diff --git a/.github/workflows/spark_sql_test.yml b/.github/workflows/spark_sql_test.yml index b8502fc01..022b98f4c 100644 --- a/.github/workflows/spark_sql_test.yml +++ b/.github/workflows/spark_sql_test.yml @@ -43,7 +43,7 @@ jobs: spark-sql-catalyst: strategy: matrix: - os: [ubuntu-latest] + os: [ubuntu-24.04] java-version: [11] spark-version: [{short: '3.4', full: '3.4.3'}, {short: '3.5', full: '3.5.1'}] module: diff --git a/.github/workflows/spark_sql_test_ansi.yml b/.github/workflows/spark_sql_test_ansi.yml index 8dda98f04..02fd56d8b 100644 --- a/.github/workflows/spark_sql_test_ansi.yml +++ b/.github/workflows/spark_sql_test_ansi.yml @@ -41,7 +41,7 @@ jobs: spark-sql-catalyst: strategy: matrix: - os: [ubuntu-latest] + os: [ubuntu-24.04] java-version: [17] spark-version: [{short: '4.0', full: '4.0.0-preview1'}] module: diff --git a/common/src/main/java/org/apache/comet/parquet/BatchReader.java b/common/src/main/java/org/apache/comet/parquet/BatchReader.java index 11c6d14dc..35d3be7f4 100644 --- a/common/src/main/java/org/apache/comet/parquet/BatchReader.java +++ b/common/src/main/java/org/apache/comet/parquet/BatchReader.java @@ -43,7 +43,6 @@ import org.apache.arrow.c.CometSchemaImporter; import org.apache.arrow.memory.BufferAllocator; -import org.apache.arrow.memory.RootAllocator; import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -69,9 +68,11 @@ import org.apache.spark.sql.execution.metric.SQLMetric; import org.apache.spark.sql.types.*; import org.apache.spark.sql.vectorized.ColumnarBatch; +import org.apache.spark.sql.vectorized.CometColumnarBatch; import org.apache.spark.util.AccumulatorV2; import org.apache.comet.CometConf; +import org.apache.comet.package$; import org.apache.comet.shims.ShimBatchReader; import org.apache.comet.shims.ShimFileFormat; import org.apache.comet.vector.CometVector; @@ -96,7 +97,7 @@ */ public class BatchReader extends RecordReader implements Closeable { private static final Logger LOG = LoggerFactory.getLogger(FileReader.class); - protected static final BufferAllocator ALLOCATOR = new RootAllocator(); + protected static final BufferAllocator ALLOCATOR = package$.MODULE$.CometArrowAllocator(); private Configuration conf; private int capacity; @@ -114,7 +115,7 @@ public class BatchReader extends RecordReader implements Cl private CometVector[] vectors; private AbstractColumnReader[] columnReaders; private CometSchemaImporter importer; - private ColumnarBatch currentBatch; + private CometColumnarBatch currentBatch; private Future> prefetchTask; private LinkedBlockingQueue> prefetchQueue; private FileReader fileReader; @@ -195,7 +196,7 @@ public BatchReader(AbstractColumnReader[] columnReaders) { int numColumns = columnReaders.length; this.columnReaders = new AbstractColumnReader[numColumns]; vectors = new CometVector[numColumns]; - currentBatch = new ColumnarBatch(vectors); + currentBatch = new CometColumnarBatch(vectors); // This constructor is used by Iceberg only. The columnReaders are // initialized in Iceberg, so no need to call the init() isInitialized = true; @@ -340,7 +341,7 @@ public void init() throws URISyntaxException, IOException { } vectors = new CometVector[numColumns]; - currentBatch = new ColumnarBatch(vectors); + currentBatch = new CometColumnarBatch(vectors); fileReader.setRequestedSchema(requestedSchema.getColumns()); // For test purpose only diff --git a/common/src/main/java/org/apache/comet/parquet/MetadataColumnReader.java b/common/src/main/java/org/apache/comet/parquet/MetadataColumnReader.java index b8722ca78..1a2570706 100644 --- a/common/src/main/java/org/apache/comet/parquet/MetadataColumnReader.java +++ b/common/src/main/java/org/apache/comet/parquet/MetadataColumnReader.java @@ -23,17 +23,17 @@ import org.apache.arrow.c.ArrowSchema; import org.apache.arrow.c.Data; import org.apache.arrow.memory.BufferAllocator; -import org.apache.arrow.memory.RootAllocator; import org.apache.arrow.vector.FieldVector; import org.apache.parquet.column.ColumnDescriptor; import org.apache.spark.sql.types.DataType; +import org.apache.comet.package$; import org.apache.comet.vector.CometPlainVector; import org.apache.comet.vector.CometVector; /** A metadata column reader that can be extended by {@link RowIndexColumnReader} etc. */ public class MetadataColumnReader extends AbstractColumnReader { - private final BufferAllocator allocator = new RootAllocator(); + private final BufferAllocator allocator = package$.MODULE$.CometArrowAllocator(); private CometVector vector; public MetadataColumnReader(DataType type, ColumnDescriptor descriptor, boolean useDecimal128) { diff --git a/common/src/main/java/org/apache/spark/sql/vectorized/CometColumnarBatch.java b/common/src/main/java/org/apache/spark/sql/vectorized/CometColumnarBatch.java new file mode 100644 index 000000000..65810c87c --- /dev/null +++ b/common/src/main/java/org/apache/spark/sql/vectorized/CometColumnarBatch.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.vectorized; + +public class CometColumnarBatch extends ColumnarBatch { + /** + * This special `ColumnarBatch` is only used by Comet `BatchReader` which reuses the same + * `ColumnVector`s for each batch if possible. + */ + @Override + public void close() { + // no-op + } + + public CometColumnarBatch(ColumnVector[] columns) { + this(columns, 0); + } + + /** + * Create a new batch from existing column vectors. + * + * @param columns The columns of this batch + * @param numRows The number of rows in this batch + */ + public CometColumnarBatch(ColumnVector[] columns, int numRows) { + super(columns, numRows); + } +} diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 1db802987..56886a1dc 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -204,17 +204,6 @@ object CometConf extends ShimCometConf { .checkValues(Set("native", "jvm", "auto")) .createWithDefault("jvm") - val COMET_SHUFFLE_ENFORCE_MODE_ENABLED: ConfigEntry[Boolean] = - conf("spark.comet.shuffle.enforceMode.enabled") - .doc( - "Comet shuffle doesn't support Spark AQE coalesce partitions. If AQE coalesce " + - "partitions is enabled, Comet shuffle won't be triggered even enabled. This config " + - "is used to enforce Comet to trigger shuffle even if AQE coalesce partitions is " + - "enabled. This is for testing purpose only.") - .internal() - .booleanConf - .createWithDefault(false) - val COMET_EXEC_BROADCAST_FORCE_ENABLED: ConfigEntry[Boolean] = conf(s"$COMET_EXEC_CONFIG_PREFIX.broadcast.enabled") .doc( diff --git a/common/src/main/scala/org/apache/spark/sql/comet/execution/arrow/CometArrowConverters.scala b/common/src/main/scala/org/apache/spark/sql/comet/execution/arrow/CometArrowConverters.scala index 9dbd8dcdf..be4efc487 100644 --- a/common/src/main/scala/org/apache/spark/sql/comet/execution/arrow/CometArrowConverters.scala +++ b/common/src/main/scala/org/apache/spark/sql/comet/execution/arrow/CometArrowConverters.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.comet.execution.arrow -import org.apache.arrow.memory.{BufferAllocator, RootAllocator} +import org.apache.arrow.memory.BufferAllocator import org.apache.arrow.vector.VectorSchemaRoot import org.apache.spark.TaskContext import org.apache.spark.internal.Logging @@ -28,11 +28,11 @@ import org.apache.spark.sql.comet.util.Utils import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.comet.CometArrowAllocator import org.apache.comet.vector.NativeUtil object CometArrowConverters extends Logging { - // TODO: we should reuse the same root allocator in the comet code base? - val rootAllocator: BufferAllocator = new RootAllocator(Long.MaxValue) + val rootAllocator: BufferAllocator = CometArrowAllocator // This is similar how Spark converts internal row to Arrow format except that it is transforming // the result batch to Comet's ColumnarBatch instead of serialized bytes. diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index c2a748897..8cf72fb42 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -271,10 +271,15 @@ index 56e9520fdab..917932336df 100644 spark.range(100).write.saveAsTable(s"$dbName.$table2Name") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala -index a9f69ab28a1..4056f14c893 100644 +index a9f69ab28a1..5d9d4f2cb83 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala -@@ -43,7 +43,7 @@ import org.apache.spark.sql.connector.FakeV2Provider +@@ -39,11 +39,12 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, Attri + import org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation + import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LeafNode, LocalRelation, LogicalPlan, OneRowRelation, Statistics} + import org.apache.spark.sql.catalyst.util.DateTimeUtils ++import org.apache.spark.sql.comet.CometBroadcastExchangeExec + import org.apache.spark.sql.connector.FakeV2Provider import org.apache.spark.sql.execution.{FilterExec, LogicalRDD, QueryExecution, SortExec, WholeStageCodegenExec} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.aggregate.HashAggregateExec @@ -283,7 +288,7 @@ index a9f69ab28a1..4056f14c893 100644 import org.apache.spark.sql.expressions.{Aggregator, Window} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -@@ -1981,7 +1981,7 @@ class DataFrameSuite extends QueryTest +@@ -1981,7 +1982,7 @@ class DataFrameSuite extends QueryTest fail("Should not have back to back Aggregates") } atFirstAgg = true @@ -292,7 +297,7 @@ index a9f69ab28a1..4056f14c893 100644 case _ => } } -@@ -2305,7 +2305,7 @@ class DataFrameSuite extends QueryTest +@@ -2305,7 +2306,7 @@ class DataFrameSuite extends QueryTest checkAnswer(join, df) assert( collect(join.queryExecution.executedPlan) { @@ -301,7 +306,7 @@ index a9f69ab28a1..4056f14c893 100644 assert( collect(join.queryExecution.executedPlan) { case e: ReusedExchangeExec => true }.size === 1) val broadcasted = broadcast(join) -@@ -2313,7 +2313,7 @@ class DataFrameSuite extends QueryTest +@@ -2313,10 +2314,12 @@ class DataFrameSuite extends QueryTest checkAnswer(join2, df) assert( collect(join2.queryExecution.executedPlan) { @@ -309,8 +314,14 @@ index a9f69ab28a1..4056f14c893 100644 + case _: ShuffleExchangeLike => true }.size == 1) assert( collect(join2.queryExecution.executedPlan) { - case e: BroadcastExchangeExec => true }.size === 1) -@@ -2876,7 +2876,7 @@ class DataFrameSuite extends QueryTest +- case e: BroadcastExchangeExec => true }.size === 1) ++ case e: BroadcastExchangeExec => true ++ case _: CometBroadcastExchangeExec => true ++ }.size === 1) + assert( + collect(join2.queryExecution.executedPlan) { case e: ReusedExchangeExec => true }.size == 4) + } +@@ -2876,7 +2879,7 @@ class DataFrameSuite extends QueryTest // Assert that no extra shuffle introduced by cogroup. val exchanges = collect(df3.queryExecution.executedPlan) { @@ -319,7 +330,7 @@ index a9f69ab28a1..4056f14c893 100644 } assert(exchanges.size == 2) } -@@ -3325,7 +3325,8 @@ class DataFrameSuite extends QueryTest +@@ -3325,7 +3328,8 @@ class DataFrameSuite extends QueryTest assert(df2.isLocal) } @@ -352,7 +363,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..10702dba7a9 100644 +index f33432ddb6f..8690c859d0c 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 @@ -403,7 +414,17 @@ index f33432ddb6f..10702dba7a9 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { withTable("fact", "dim") { spark.range(100).select( -@@ -1187,7 +1194,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1027,7 +1034,8 @@ abstract class DynamicPartitionPruningSuiteBase + } + } + +- test("avoid reordering broadcast join keys to match input hash partitioning") { ++ test("avoid reordering broadcast join keys to match input hash partitioning", ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + withTable("large", "dimTwo", "dimThree") { +@@ -1187,7 +1195,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -413,7 +434,7 @@ index f33432ddb6f..10702dba7a9 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( """ -@@ -1238,7 +1246,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1238,7 +1247,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -423,7 +444,7 @@ index f33432ddb6f..10702dba7a9 100644 Given("dynamic pruning filter on the build side") withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( -@@ -1279,7 +1288,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1279,7 +1289,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -433,7 +454,7 @@ index f33432ddb6f..10702dba7a9 100644 Seq(NO_CODEGEN, CODEGEN_ONLY).foreach { mode => Seq(true, false).foreach { pruning => withSQLConf( -@@ -1311,7 +1321,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1311,7 +1322,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -443,7 +464,17 @@ index f33432ddb6f..10702dba7a9 100644 withSQLConf( SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true", -@@ -1470,7 +1481,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1423,7 +1435,8 @@ abstract class DynamicPartitionPruningSuiteBase + } + } + +- test("SPARK-34637: DPP side broadcast query stage is created firstly") { ++ test("SPARK-34637: DPP side broadcast query stage is created firstly", ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { + val df = sql( + """ WITH v as ( +@@ -1470,7 +1483,8 @@ abstract class DynamicPartitionPruningSuiteBase checkAnswer(df, Row(3, 2) :: Row(3, 2) :: Row(3, 2) :: Row(3, 2) :: Nil) } @@ -453,7 +484,7 @@ index f33432ddb6f..10702dba7a9 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { val df = sql( """ -@@ -1485,7 +1497,7 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1485,7 +1499,7 @@ abstract class DynamicPartitionPruningSuiteBase } test("SPARK-38148: Do not add dynamic partition pruning if there exists static partition " + @@ -462,7 +493,7 @@ index f33432ddb6f..10702dba7a9 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { Seq( "f.store_id = 1" -> false, -@@ -1557,7 +1569,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1557,7 +1571,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -472,7 +503,7 @@ index f33432ddb6f..10702dba7a9 100644 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 +1601,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1588,7 +1603,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -482,7 +513,7 @@ index f33432ddb6f..10702dba7a9 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { val df = sql( """ -@@ -1617,7 +1631,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1617,7 +1633,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -492,7 +523,7 @@ index f33432ddb6f..10702dba7a9 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { val df = sql( """ -@@ -1729,6 +1744,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1729,6 +1746,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"))) @@ -648,17 +679,19 @@ index 1792b4c32eb..1616e6f39bd 100644 assert(shuffleMergeJoins.size == 1) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala -index 7f062bfb899..400e939468f 100644 +index 7f062bfb899..3acf697df9a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala -@@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier +@@ -30,7 +30,8 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.expressions.{Ascending, GenericRow, SortOrder} import org.apache.spark.sql.catalyst.plans.logical.Filter +-import org.apache.spark.sql.execution.{BinaryExecNode, FilterExec, ProjectExec, SortExec, SparkPlan, WholeStageCodegenExec} +import org.apache.spark.sql.comet._ - import org.apache.spark.sql.execution.{BinaryExecNode, FilterExec, ProjectExec, SortExec, SparkPlan, WholeStageCodegenExec} ++import org.apache.spark.sql.execution.{BinaryExecNode, ColumnarToRowExec, FilterExec, InputAdapter, ProjectExec, SortExec, SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.exchange.{ShuffleExchangeExec, ShuffleExchangeLike} + import org.apache.spark.sql.execution.joins._ @@ -740,7 +741,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan } } @@ -669,7 +702,20 @@ index 7f062bfb899..400e939468f 100644 withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1", SQLConf.SORT_MERGE_JOIN_EXEC_BUFFER_IN_MEMORY_THRESHOLD.key -> "0", SQLConf.SORT_MERGE_JOIN_EXEC_BUFFER_SPILL_THRESHOLD.key -> "1") { -@@ -1115,9 +1117,11 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -866,10 +868,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + val physical = df.queryExecution.sparkPlan + val physicalJoins = physical.collect { + case j: SortMergeJoinExec => j ++ case j: CometSortMergeJoinExec => j.originalPlan.asInstanceOf[SortMergeJoinExec] + } + val executed = df.queryExecution.executedPlan + val executedJoins = collect(executed) { + case j: SortMergeJoinExec => j ++ case j: CometSortMergeJoinExec => j.originalPlan.asInstanceOf[SortMergeJoinExec] + } + // This only applies to the above tested queries, in which a child SortMergeJoin always + // contains the SortOrder required by its parent SortMergeJoin. Thus, SortExec should never +@@ -1115,9 +1119,11 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan val plan = df1.join(df2.hint("SHUFFLE_HASH"), $"k1" === $"k2", joinType) .groupBy($"k1").count() .queryExecution.executedPlan @@ -683,7 +729,7 @@ index 7f062bfb899..400e939468f 100644 }) } -@@ -1134,10 +1138,11 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1134,10 +1140,11 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan .join(df4.hint("SHUFFLE_MERGE"), $"k1" === $"k4", joinType) .queryExecution .executedPlan @@ -697,7 +743,7 @@ index 7f062bfb899..400e939468f 100644 }) // Test shuffled hash join -@@ -1147,10 +1152,13 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1147,10 +1154,13 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan .join(df4.hint("SHUFFLE_MERGE"), $"k1" === $"k4", joinType) .queryExecution .executedPlan @@ -714,7 +760,7 @@ index 7f062bfb899..400e939468f 100644 }) } -@@ -1241,12 +1249,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1241,12 +1251,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan inputDFs.foreach { case (df1, df2, joinExprs) => val smjDF = df1.join(df2.hint("SHUFFLE_MERGE"), joinExprs, "full") assert(collect(smjDF.queryExecution.executedPlan) { @@ -729,7 +775,35 @@ index 7f062bfb899..400e939468f 100644 // Same result between shuffled hash join and sort merge join checkAnswer(shjDF, smjResult) } -@@ -1341,7 +1349,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1282,18 +1292,25 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + } + + // Test shuffled hash join +- withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { ++ withSQLConf("spark.comet.enabled" -> "true", ++ SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + val shjCodegenDF = df1.join(df2.hint("SHUFFLE_HASH"), $"k1" === $"k2", joinType) + assert(shjCodegenDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_ : ShuffledHashJoinExec) => true + case WholeStageCodegenExec(ProjectExec(_, _ : ShuffledHashJoinExec)) => true ++ case WholeStageCodegenExec(ColumnarToRowExec(InputAdapter(_: CometHashJoinExec))) => ++ true ++ case WholeStageCodegenExec(ColumnarToRowExec( ++ InputAdapter(CometProjectExec(_, _, _, _, _: CometHashJoinExec, _)))) => true + }.size === 1) + checkAnswer(shjCodegenDF, Seq.empty) + + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { + val shjNonCodegenDF = df1.join(df2.hint("SHUFFLE_HASH"), $"k1" === $"k2", joinType) + assert(shjNonCodegenDF.queryExecution.executedPlan.collect { +- case _: ShuffledHashJoinExec => true }.size === 1) ++ case _: ShuffledHashJoinExec => true ++ case _: CometHashJoinExec => true ++ }.size === 1) + checkAnswer(shjNonCodegenDF, Seq.empty) + } + } +@@ -1341,7 +1358,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan val plan = sql(getAggQuery(selectExpr, joinType)).queryExecution.executedPlan assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) // Have shuffle before aggregation @@ -739,7 +813,7 @@ index 7f062bfb899..400e939468f 100644 } def getJoinQuery(selectExpr: String, joinType: String): String = { -@@ -1370,9 +1379,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1370,9 +1388,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan } val plan = sql(getJoinQuery(selectExpr, joinType)).queryExecution.executedPlan assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) @@ -754,7 +828,7 @@ index 7f062bfb899..400e939468f 100644 } // Test output ordering is not preserved -@@ -1381,9 +1393,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1381,9 +1402,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan val selectExpr = "/*+ BROADCAST(left_t) */ k1 as k0" val plan = sql(getJoinQuery(selectExpr, joinType)).queryExecution.executedPlan assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) @@ -769,7 +843,7 @@ index 7f062bfb899..400e939468f 100644 } // Test singe partition -@@ -1393,7 +1408,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1393,7 +1417,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan |FROM range(0, 10, 1, 1) t1 FULL OUTER JOIN range(0, 10, 1, 1) t2 |""".stripMargin) val plan = fullJoinDF.queryExecution.executedPlan @@ -779,6 +853,38 @@ index 7f062bfb899..400e939468f 100644 checkAnswer(fullJoinDF, Row(100)) } } +@@ -1438,6 +1463,9 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + Seq(semiJoinDF, antiJoinDF).foreach { df => + assert(collect(df.queryExecution.executedPlan) { + case j: ShuffledHashJoinExec if j.ignoreDuplicatedKey == ignoreDuplicatedKey => true ++ case j: CometHashJoinExec ++ if j.originalPlan.asInstanceOf[ShuffledHashJoinExec].ignoreDuplicatedKey == ++ ignoreDuplicatedKey => true + }.size == 1) + } + } +@@ -1489,7 +1517,10 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + + test("SPARK-43113: Full outer join with duplicate stream-side references in condition (SHJ)") { + def check(plan: SparkPlan): Unit = { +- assert(collect(plan) { case _: ShuffledHashJoinExec => true }.size === 1) ++ assert(collect(plan) { ++ case _: ShuffledHashJoinExec => true ++ case _: CometHashJoinExec => true ++ }.size === 1) + } + dupStreamSideColTest("SHUFFLE_HASH", check) + } +@@ -1605,7 +1636,8 @@ class ThreadLeakInSortMergeJoinSuite + sparkConf.set(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD, 20)) + } + +- test("SPARK-47146: thread leak when doing SortMergeJoin (with spill)") { ++ test("SPARK-47146: thread leak when doing SortMergeJoin (with spill)", ++ IgnoreComet("Comet SMJ doesn't spill yet")) { + + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala index b5b34922694..a72403780c4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala @@ -1134,26 +1240,86 @@ index 47679ed7865..9ffbaecb98e 100644 assert(collectWithSubqueries(plan) { case s: SortAggregateExec => s }.length == sortAggCount) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala -index ac710c32296..e163c1a6a76 100644 +index ac710c32296..baae214c6ee 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.{Dataset, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.expressions.codegen.{ByteCodeStats, CodeAndComment, CodeGenerator} -+import org.apache.spark.sql.comet.CometSortMergeJoinExec ++import org.apache.spark.sql.comet.{CometHashJoinExec, CometSortMergeJoinExec} import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecutionSuite import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, SortAggregateExec} import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec -@@ -224,6 +225,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession +@@ -169,6 +170,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + val oneJoinDF = df1.join(df2.hint("SHUFFLE_HASH"), $"k1" === $"k2") + assert(oneJoinDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_ : ShuffledHashJoinExec) => true ++ case _: CometHashJoinExec => true + }.size === 1) + checkAnswer(oneJoinDF, Seq(Row(0, 0), Row(1, 1), Row(2, 2), Row(3, 3), Row(4, 4))) + +@@ -177,6 +179,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + .join(df3.hint("SHUFFLE_HASH"), $"k1" === $"k3") + assert(twoJoinsDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_ : ShuffledHashJoinExec) => true ++ case _: CometHashJoinExec => true + }.size === 2) + checkAnswer(twoJoinsDF, + Seq(Row(0, 0, 0), Row(1, 1, 1), Row(2, 2, 2), Row(3, 3, 3), Row(4, 4, 4))) +@@ -193,6 +196,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + assert(joinUniqueDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_ : ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true + case WholeStageCodegenExec(_ : SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true ++ case _: CometHashJoinExec if hint == "SHUFFLE_HASH" => true ++ case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true + }.size === 1) + checkAnswer(joinUniqueDF, Seq(Row(0, 0), Row(1, 1), Row(2, 2), Row(3, 3), Row(4, 4), + Row(null, 5), Row(null, 6), Row(null, 7), Row(null, 8), Row(null, 9))) +@@ -203,6 +208,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + assert(joinNonUniqueDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_ : ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true + case WholeStageCodegenExec(_ : SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true ++ case _: CometHashJoinExec if hint == "SHUFFLE_HASH" => true ++ case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true + }.size === 1) + checkAnswer(joinNonUniqueDF, Seq(Row(0, 0), Row(0, 3), Row(0, 6), Row(0, 9), Row(1, 1), + Row(1, 4), Row(1, 7), Row(2, 2), Row(2, 5), Row(2, 8), Row(3, null), Row(4, null))) +@@ -213,6 +220,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + assert(joinWithNonEquiDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_ : ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true + case WholeStageCodegenExec(_ : SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true ++ case _: CometHashJoinExec if hint == "SHUFFLE_HASH" => true ++ case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true + }.size === 1) + checkAnswer(joinWithNonEquiDF, Seq(Row(0, 0), Row(0, 6), Row(0, 9), Row(1, 1), + Row(1, 7), Row(2, 2), Row(2, 8), Row(3, null), Row(4, null), Row(null, 3), Row(null, 4), +@@ -224,6 +233,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession assert(twoJoinsDF.queryExecution.executedPlan.collect { case WholeStageCodegenExec(_ : ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true case WholeStageCodegenExec(_ : SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true ++ case _: CometHashJoinExec if hint == "SHUFFLE_HASH" => true + case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true }.size === 2) checkAnswer(twoJoinsDF, Seq(Row(0, 0, 0), Row(1, 1, null), Row(2, 2, 2), Row(3, 3, null), Row(4, 4, null), -@@ -258,6 +260,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession +@@ -241,6 +252,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + val oneLeftOuterJoinDF = df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", "left_outer") + assert(oneLeftOuterJoinDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_ : SortMergeJoinExec) => true ++ case _: CometSortMergeJoinExec => true + }.size === 1) + checkAnswer(oneLeftOuterJoinDF, Seq(Row(0, 0), Row(1, 1), Row(2, 2), Row(3, 3), Row(4, null), + Row(5, null), Row(6, null), Row(7, null), Row(8, null), Row(9, null))) +@@ -249,6 +261,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + val oneRightOuterJoinDF = df2.join(df3.hint("SHUFFLE_MERGE"), $"k2" === $"k3", "right_outer") + assert(oneRightOuterJoinDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_ : SortMergeJoinExec) => true ++ case _: CometSortMergeJoinExec => true + }.size === 1) + checkAnswer(oneRightOuterJoinDF, Seq(Row(0, 0), Row(1, 1), Row(2, 2), Row(3, 3), Row(null, 4), + Row(null, 5))) +@@ -258,6 +271,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession .join(df1.hint("SHUFFLE_MERGE"), $"k3" === $"k1", "right_outer") assert(twoJoinsDF.queryExecution.executedPlan.collect { case WholeStageCodegenExec(_ : SortMergeJoinExec) => true @@ -1161,38 +1327,56 @@ index ac710c32296..e163c1a6a76 100644 }.size === 2) checkAnswer(twoJoinsDF, Seq(Row(0, 0, 0), Row(1, 1, 1), Row(2, 2, 2), Row(3, 3, 3), Row(4, null, 4), Row(5, null, 5), -@@ -280,8 +283,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession +@@ -273,6 +287,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + val oneJoinDF = df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", "left_semi") + assert(oneJoinDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(ProjectExec(_, _ : SortMergeJoinExec)) => true ++ case _: CometSortMergeJoinExec => true + }.size === 1) + checkAnswer(oneJoinDF, Seq(Row(0), Row(1), Row(2), Row(3))) + +@@ -280,8 +295,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession val twoJoinsDF = df3.join(df2.hint("SHUFFLE_MERGE"), $"k3" === $"k2", "left_semi") .join(df1.hint("SHUFFLE_MERGE"), $"k3" === $"k1", "left_semi") assert(twoJoinsDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(ProjectExec(_, _ : SortMergeJoinExec)) | - WholeStageCodegenExec(_ : SortMergeJoinExec) => true + case _: SortMergeJoinExec => true ++ case _: CometSortMergeJoinExec => true }.size === 2) checkAnswer(twoJoinsDF, Seq(Row(0), Row(1), Row(2), Row(3))) } -@@ -302,8 +304,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession +@@ -295,6 +310,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + val oneJoinDF = df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", "left_anti") + assert(oneJoinDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(ProjectExec(_, _ : SortMergeJoinExec)) => true ++ case _: CometSortMergeJoinExec => true + }.size === 1) + checkAnswer(oneJoinDF, Seq(Row(4), Row(5), Row(6), Row(7), Row(8), Row(9))) + +@@ -302,8 +318,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession val twoJoinsDF = df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", "left_anti") .join(df3.hint("SHUFFLE_MERGE"), $"k1" === $"k3", "left_anti") assert(twoJoinsDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(ProjectExec(_, _ : SortMergeJoinExec)) | - WholeStageCodegenExec(_ : SortMergeJoinExec) => true + case _: SortMergeJoinExec => true ++ case _: CometSortMergeJoinExec => true }.size === 2) checkAnswer(twoJoinsDF, Seq(Row(6), Row(7), Row(8), Row(9))) } -@@ -436,7 +437,9 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - val plan = df.queryExecution.executedPlan - assert(plan.exists(p => - p.isInstanceOf[WholeStageCodegenExec] && +@@ -433,10 +449,6 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + + test("Sort should be included in WholeStageCodegen") { + val df = spark.range(3, 0, -1).toDF().sort(col("id")) +- val plan = df.queryExecution.executedPlan +- assert(plan.exists(p => +- p.isInstanceOf[WholeStageCodegenExec] && - p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[SortExec])) -+ p.asInstanceOf[WholeStageCodegenExec].collect { -+ case _: SortExec => true -+ }.nonEmpty)) assert(df.collect() === Array(Row(1), Row(2), Row(3))) } -@@ -616,7 +619,9 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession +@@ -616,7 +628,9 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession .write.mode(SaveMode.Overwrite).parquet(path) withSQLConf(SQLConf.WHOLESTAGE_MAX_NUM_FIELDS.key -> "255", @@ -1204,7 +1388,7 @@ index ac710c32296..e163c1a6a76 100644 val df = spark.read.parquet(path).selectExpr(projection: _*) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala -index 593bd7bb4ba..7ad55e3ab20 100644 +index 593bd7bb4ba..32af28b0238 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -26,9 +26,11 @@ import org.scalatest.time.SpanSugar._ @@ -1228,7 +1412,7 @@ index 593bd7bb4ba..7ad55e3ab20 100644 } } -@@ -116,30 +119,38 @@ class AdaptiveQueryExecSuite +@@ -116,30 +119,39 @@ class AdaptiveQueryExecSuite private def findTopLevelSortMergeJoin(plan: SparkPlan): Seq[SortMergeJoinExec] = { collect(plan) { case j: SortMergeJoinExec => j @@ -1250,6 +1434,7 @@ index 593bd7bb4ba..7ad55e3ab20 100644 case j: BaseJoinExec => j + case c: CometHashJoinExec => c.originalPlan.asInstanceOf[BaseJoinExec] + case c: CometSortMergeJoinExec => c.originalPlan.asInstanceOf[BaseJoinExec] ++ case c: CometBroadcastHashJoinExec => c.originalPlan.asInstanceOf[BaseJoinExec] } } @@ -1267,7 +1452,7 @@ index 593bd7bb4ba..7ad55e3ab20 100644 } } -@@ -176,6 +187,7 @@ class AdaptiveQueryExecSuite +@@ -176,6 +188,7 @@ class AdaptiveQueryExecSuite val parts = rdd.partitions assert(parts.forall(rdd.preferredLocations(_).nonEmpty)) } @@ -1275,7 +1460,7 @@ index 593bd7bb4ba..7ad55e3ab20 100644 assert(numShuffles === (numLocalReads.length + numShufflesWithoutLocalRead)) } -@@ -184,7 +196,7 @@ class AdaptiveQueryExecSuite +@@ -184,7 +197,7 @@ class AdaptiveQueryExecSuite val plan = df.queryExecution.executedPlan assert(plan.isInstanceOf[AdaptiveSparkPlanExec]) val shuffle = plan.asInstanceOf[AdaptiveSparkPlanExec].executedPlan.collect { @@ -1284,7 +1469,7 @@ index 593bd7bb4ba..7ad55e3ab20 100644 } assert(shuffle.size == 1) assert(shuffle(0).outputPartitioning.numPartitions == numPartition) -@@ -200,7 +212,8 @@ class AdaptiveQueryExecSuite +@@ -200,7 +213,8 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) @@ -1294,7 +1479,7 @@ index 593bd7bb4ba..7ad55e3ab20 100644 } } -@@ -227,7 +240,8 @@ class AdaptiveQueryExecSuite +@@ -227,7 +241,8 @@ class AdaptiveQueryExecSuite } } @@ -1304,7 +1489,7 @@ index 593bd7bb4ba..7ad55e3ab20 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80", -@@ -259,7 +273,8 @@ class AdaptiveQueryExecSuite +@@ -259,7 +274,8 @@ class AdaptiveQueryExecSuite } } @@ -1314,7 +1499,7 @@ index 593bd7bb4ba..7ad55e3ab20 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80", -@@ -273,7 +288,8 @@ class AdaptiveQueryExecSuite +@@ -273,7 +289,8 @@ class AdaptiveQueryExecSuite val localReads = collect(adaptivePlan) { case read: AQEShuffleReadExec if read.isLocalRead => read } @@ -1324,7 +1509,29 @@ index 593bd7bb4ba..7ad55e3ab20 100644 val localShuffleRDD0 = localReads(0).execute().asInstanceOf[ShuffledRowRDD] val localShuffleRDD1 = localReads(1).execute().asInstanceOf[ShuffledRowRDD] // the final parallelism is math.max(1, numReduces / numMappers): math.max(1, 5/2) = 2 -@@ -322,7 +338,7 @@ class AdaptiveQueryExecSuite +@@ -298,7 +315,9 @@ class AdaptiveQueryExecSuite + .groupBy($"a").count() + checkAnswer(testDf, Seq()) + val plan = testDf.queryExecution.executedPlan +- assert(find(plan)(_.isInstanceOf[SortMergeJoinExec]).isDefined) ++ assert(find(plan) { case p => ++ p.isInstanceOf[SortMergeJoinExec] || p.isInstanceOf[CometSortMergeJoinExec] ++ }.isDefined) + val coalescedReads = collect(plan) { + case r: AQEShuffleReadExec => r + } +@@ -312,7 +331,9 @@ class AdaptiveQueryExecSuite + .groupBy($"a").count() + checkAnswer(testDf, Seq()) + val plan = testDf.queryExecution.executedPlan +- assert(find(plan)(_.isInstanceOf[BroadcastHashJoinExec]).isDefined) ++ assert(find(plan) { case p => ++ p.isInstanceOf[BroadcastHashJoinExec] || p.isInstanceOf[CometBroadcastHashJoinExec] ++ }.isDefined) + val coalescedReads = collect(plan) { + case r: AQEShuffleReadExec => r + } +@@ -322,7 +343,7 @@ class AdaptiveQueryExecSuite } } @@ -1333,7 +1540,7 @@ index 593bd7bb4ba..7ad55e3ab20 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { -@@ -337,7 +353,7 @@ class AdaptiveQueryExecSuite +@@ -337,7 +358,7 @@ class AdaptiveQueryExecSuite } } @@ -1342,7 +1549,7 @@ index 593bd7bb4ba..7ad55e3ab20 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { -@@ -353,7 +369,7 @@ class AdaptiveQueryExecSuite +@@ -353,7 +374,7 @@ class AdaptiveQueryExecSuite } } @@ -1351,7 +1558,7 @@ index 593bd7bb4ba..7ad55e3ab20 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { -@@ -398,7 +414,7 @@ class AdaptiveQueryExecSuite +@@ -398,7 +419,7 @@ class AdaptiveQueryExecSuite } } @@ -1360,7 +1567,7 @@ index 593bd7bb4ba..7ad55e3ab20 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { -@@ -443,7 +459,7 @@ class AdaptiveQueryExecSuite +@@ -443,7 +464,7 @@ class AdaptiveQueryExecSuite } } @@ -1369,7 +1576,16 @@ index 593bd7bb4ba..7ad55e3ab20 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "500") { -@@ -508,7 +524,7 @@ class AdaptiveQueryExecSuite +@@ -489,7 +510,7 @@ class AdaptiveQueryExecSuite + } + } + +- test("Exchange reuse") { ++ test("Exchange reuse", IgnoreComet("Comet shuffle changes shuffle metrics")) { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { +@@ -508,7 +529,7 @@ class AdaptiveQueryExecSuite } } @@ -1378,7 +1594,7 @@ index 593bd7bb4ba..7ad55e3ab20 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { -@@ -539,7 +555,9 @@ class AdaptiveQueryExecSuite +@@ -539,7 +560,9 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) @@ -1389,7 +1605,7 @@ index 593bd7bb4ba..7ad55e3ab20 100644 // Even with local shuffle read, the query stage reuse can also work. val ex = findReusedExchange(adaptivePlan) assert(ex.nonEmpty) -@@ -560,7 +578,9 @@ class AdaptiveQueryExecSuite +@@ -560,7 +583,9 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) @@ -1400,7 +1616,7 @@ index 593bd7bb4ba..7ad55e3ab20 100644 // Even with local shuffle read, the query stage reuse can also work. val ex = findReusedExchange(adaptivePlan) assert(ex.isEmpty) -@@ -569,7 +589,8 @@ class AdaptiveQueryExecSuite +@@ -569,7 +594,8 @@ class AdaptiveQueryExecSuite } } @@ -1410,7 +1626,7 @@ index 593bd7bb4ba..7ad55e3ab20 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "20000000", -@@ -664,7 +685,8 @@ class AdaptiveQueryExecSuite +@@ -664,7 +690,8 @@ class AdaptiveQueryExecSuite val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) // There is still a SMJ, and its two shuffles can't apply local read. @@ -1420,7 +1636,7 @@ index 593bd7bb4ba..7ad55e3ab20 100644 } } -@@ -786,7 +808,8 @@ class AdaptiveQueryExecSuite +@@ -786,7 +813,8 @@ class AdaptiveQueryExecSuite } } @@ -1430,7 +1646,7 @@ index 593bd7bb4ba..7ad55e3ab20 100644 Seq("SHUFFLE_MERGE", "SHUFFLE_HASH").foreach { joinHint => def getJoinNode(plan: SparkPlan): Seq[ShuffledJoin] = if (joinHint == "SHUFFLE_MERGE") { findTopLevelSortMergeJoin(plan) -@@ -1004,7 +1027,8 @@ class AdaptiveQueryExecSuite +@@ -1004,7 +1032,8 @@ class AdaptiveQueryExecSuite } } @@ -1440,7 +1656,7 @@ index 593bd7bb4ba..7ad55e3ab20 100644 withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { val (_, adaptivePlan) = runAdaptiveAndVerifyResult( "SELECT key FROM testData GROUP BY key") -@@ -1599,7 +1623,7 @@ class AdaptiveQueryExecSuite +@@ -1599,7 +1628,7 @@ class AdaptiveQueryExecSuite val (_, adaptivePlan) = runAdaptiveAndVerifyResult( "SELECT id FROM v1 GROUP BY id DISTRIBUTE BY id") assert(collect(adaptivePlan) { @@ -1449,7 +1665,7 @@ index 593bd7bb4ba..7ad55e3ab20 100644 }.length == 1) } } -@@ -1679,7 +1703,8 @@ class AdaptiveQueryExecSuite +@@ -1679,7 +1708,8 @@ class AdaptiveQueryExecSuite } } @@ -1459,7 +1675,7 @@ index 593bd7bb4ba..7ad55e3ab20 100644 def hasRepartitionShuffle(plan: SparkPlan): Boolean = { find(plan) { case s: ShuffleExchangeLike => -@@ -1864,6 +1889,9 @@ class AdaptiveQueryExecSuite +@@ -1864,6 +1894,9 @@ class AdaptiveQueryExecSuite def checkNoCoalescePartitions(ds: Dataset[Row], origin: ShuffleOrigin): Unit = { assert(collect(ds.queryExecution.executedPlan) { case s: ShuffleExchangeExec if s.shuffleOrigin == origin && s.numPartitions == 2 => s @@ -1469,7 +1685,7 @@ index 593bd7bb4ba..7ad55e3ab20 100644 }.size == 1) ds.collect() val plan = ds.queryExecution.executedPlan -@@ -1872,6 +1900,9 @@ class AdaptiveQueryExecSuite +@@ -1872,6 +1905,9 @@ class AdaptiveQueryExecSuite }.isEmpty) assert(collect(plan) { case s: ShuffleExchangeExec if s.shuffleOrigin == origin && s.numPartitions == 2 => s @@ -1479,7 +1695,7 @@ index 593bd7bb4ba..7ad55e3ab20 100644 }.size == 1) checkAnswer(ds, testData) } -@@ -2028,7 +2059,8 @@ class AdaptiveQueryExecSuite +@@ -2028,7 +2064,8 @@ class AdaptiveQueryExecSuite } } @@ -1489,7 +1705,7 @@ index 593bd7bb4ba..7ad55e3ab20 100644 withTempView("t1", "t2") { def checkJoinStrategy(shouldShuffleHashJoin: Boolean): Unit = { Seq("100", "100000").foreach { size => -@@ -2114,7 +2146,8 @@ class AdaptiveQueryExecSuite +@@ -2114,7 +2151,8 @@ class AdaptiveQueryExecSuite } } @@ -1499,7 +1715,7 @@ index 593bd7bb4ba..7ad55e3ab20 100644 withTempView("v") { withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", -@@ -2213,7 +2246,7 @@ class AdaptiveQueryExecSuite +@@ -2213,7 +2251,7 @@ class AdaptiveQueryExecSuite runAdaptiveAndVerifyResult(s"SELECT $repartition key1 FROM skewData1 " + s"JOIN skewData2 ON key1 = key2 GROUP BY key1") val shuffles1 = collect(adaptive1) { @@ -1508,7 +1724,7 @@ index 593bd7bb4ba..7ad55e3ab20 100644 } assert(shuffles1.size == 3) // shuffles1.head is the top-level shuffle under the Aggregate operator -@@ -2226,7 +2259,7 @@ class AdaptiveQueryExecSuite +@@ -2226,7 +2264,7 @@ class AdaptiveQueryExecSuite runAdaptiveAndVerifyResult(s"SELECT $repartition key1 FROM skewData1 " + s"JOIN skewData2 ON key1 = key2") val shuffles2 = collect(adaptive2) { @@ -1517,7 +1733,7 @@ index 593bd7bb4ba..7ad55e3ab20 100644 } if (hasRequiredDistribution) { assert(shuffles2.size == 3) -@@ -2260,7 +2293,8 @@ class AdaptiveQueryExecSuite +@@ -2260,7 +2298,8 @@ class AdaptiveQueryExecSuite } } @@ -1527,7 +1743,7 @@ index 593bd7bb4ba..7ad55e3ab20 100644 CostEvaluator.instantiate( classOf[SimpleShuffleSortCostEvaluator].getCanonicalName, spark.sparkContext.getConf) intercept[IllegalArgumentException] { -@@ -2404,6 +2438,7 @@ class AdaptiveQueryExecSuite +@@ -2404,6 +2443,7 @@ class AdaptiveQueryExecSuite val (_, adaptive) = runAdaptiveAndVerifyResult(query) assert(adaptive.collect { case sort: SortExec => sort @@ -1535,7 +1751,7 @@ index 593bd7bb4ba..7ad55e3ab20 100644 }.size == 1) val read = collect(adaptive) { case read: AQEShuffleReadExec => read -@@ -2421,7 +2456,8 @@ class AdaptiveQueryExecSuite +@@ -2421,7 +2461,8 @@ class AdaptiveQueryExecSuite } } @@ -1545,7 +1761,7 @@ index 593bd7bb4ba..7ad55e3ab20 100644 withTempView("v") { withSQLConf( SQLConf.ADAPTIVE_OPTIMIZE_SKEWS_IN_REBALANCE_PARTITIONS_ENABLED.key -> "true", -@@ -2533,7 +2569,7 @@ class AdaptiveQueryExecSuite +@@ -2533,7 +2574,7 @@ class AdaptiveQueryExecSuite runAdaptiveAndVerifyResult("SELECT key1 FROM skewData1 JOIN skewData2 ON key1 = key2 " + "JOIN skewData3 ON value2 = value3") val shuffles1 = collect(adaptive1) { @@ -1554,7 +1770,7 @@ index 593bd7bb4ba..7ad55e3ab20 100644 } assert(shuffles1.size == 4) val smj1 = findTopLevelSortMergeJoin(adaptive1) -@@ -2544,7 +2580,7 @@ class AdaptiveQueryExecSuite +@@ -2544,7 +2585,7 @@ class AdaptiveQueryExecSuite runAdaptiveAndVerifyResult("SELECT key1 FROM skewData1 JOIN skewData2 ON key1 = key2 " + "JOIN skewData3 ON value1 = value3") val shuffles2 = collect(adaptive2) { @@ -1584,10 +1800,10 @@ index bd9c79e5b96..ab7584e768e 100644 assert(fileSourceScanSchemata.size === expectedSchemaCatalogStrings.size, s"Found ${fileSourceScanSchemata.size} file sources in dataframe, " + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala -index ce43edb79c1..c414b19eda7 100644 +index ce43edb79c1..8436cb727c6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala -@@ -17,7 +17,7 @@ +@@ -17,9 +17,10 @@ package org.apache.spark.sql.execution.datasources @@ -1595,8 +1811,27 @@ index ce43edb79c1..c414b19eda7 100644 +import org.apache.spark.sql.{IgnoreComet, QueryTest, Row} import org.apache.spark.sql.catalyst.expressions.{Ascending, AttributeReference, NullsFirst, SortOrder} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Sort} ++import org.apache.spark.sql.comet.CometSortExec import org.apache.spark.sql.execution.{QueryExecution, SortExec} -@@ -305,7 +305,8 @@ class V1WriteCommandSuite extends QueryTest with SharedSparkSession with V1Write + import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec + import org.apache.spark.sql.internal.SQLConf +@@ -225,6 +226,7 @@ class V1WriteCommandSuite extends QueryTest with SharedSparkSession with V1Write + // assert the outer most sort in the executed plan + assert(plan.collectFirst { + case s: SortExec => s ++ case s: CometSortExec => s.originalPlan.asInstanceOf[SortExec] + }.exists { + case SortExec(Seq( + SortOrder(AttributeReference("key", IntegerType, _, _), Ascending, NullsFirst, _), +@@ -272,6 +274,7 @@ class V1WriteCommandSuite extends QueryTest with SharedSparkSession with V1Write + // assert the outer most sort in the executed plan + assert(plan.collectFirst { + case s: SortExec => s ++ case s: CometSortExec => s.originalPlan.asInstanceOf[SortExec] + }.exists { + case SortExec(Seq( + SortOrder(AttributeReference("value", StringType, _, _), Ascending, NullsFirst, _), +@@ -305,7 +308,8 @@ class V1WriteCommandSuite extends QueryTest with SharedSparkSession with V1Write } } diff --git a/dev/diffs/3.5.1.diff b/dev/diffs/3.5.1.diff index cb8d409c5..046e9f3ce 100644 --- a/dev/diffs/3.5.1.diff +++ b/dev/diffs/3.5.1.diff @@ -250,10 +250,15 @@ index 56e9520fdab..917932336df 100644 spark.range(100).write.saveAsTable(s"$dbName.$table2Name") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala -index 002719f0689..fa76c42feaa 100644 +index 002719f0689..784d24afe2d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala -@@ -44,7 +44,7 @@ import org.apache.spark.sql.connector.FakeV2Provider +@@ -40,11 +40,12 @@ import org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation + import org.apache.spark.sql.catalyst.parser.ParseException + import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LeafNode, LocalRelation, LogicalPlan, OneRowRelation, Statistics} + import org.apache.spark.sql.catalyst.util.DateTimeUtils ++import org.apache.spark.sql.comet.CometBroadcastExchangeExec + import org.apache.spark.sql.connector.FakeV2Provider import org.apache.spark.sql.execution.{FilterExec, LogicalRDD, QueryExecution, SortExec, WholeStageCodegenExec} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.aggregate.HashAggregateExec @@ -262,7 +267,7 @@ index 002719f0689..fa76c42feaa 100644 import org.apache.spark.sql.expressions.{Aggregator, Window} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -@@ -2020,7 +2020,7 @@ class DataFrameSuite extends QueryTest +@@ -2020,7 +2021,7 @@ class DataFrameSuite extends QueryTest fail("Should not have back to back Aggregates") } atFirstAgg = true @@ -271,7 +276,7 @@ index 002719f0689..fa76c42feaa 100644 case _ => } } -@@ -2344,7 +2344,7 @@ class DataFrameSuite extends QueryTest +@@ -2344,7 +2345,7 @@ class DataFrameSuite extends QueryTest checkAnswer(join, df) assert( collect(join.queryExecution.executedPlan) { @@ -280,7 +285,7 @@ index 002719f0689..fa76c42feaa 100644 assert( collect(join.queryExecution.executedPlan) { case e: ReusedExchangeExec => true }.size === 1) val broadcasted = broadcast(join) -@@ -2352,7 +2352,7 @@ class DataFrameSuite extends QueryTest +@@ -2352,10 +2353,12 @@ class DataFrameSuite extends QueryTest checkAnswer(join2, df) assert( collect(join2.queryExecution.executedPlan) { @@ -288,8 +293,14 @@ index 002719f0689..fa76c42feaa 100644 + case _: ShuffleExchangeLike => true }.size == 1) assert( collect(join2.queryExecution.executedPlan) { - case e: BroadcastExchangeExec => true }.size === 1) -@@ -2915,7 +2915,7 @@ class DataFrameSuite extends QueryTest +- case e: BroadcastExchangeExec => true }.size === 1) ++ case e: BroadcastExchangeExec => true ++ case _: CometBroadcastExchangeExec => true ++ }.size === 1) + assert( + collect(join2.queryExecution.executedPlan) { case e: ReusedExchangeExec => true }.size == 4) + } +@@ -2915,7 +2918,7 @@ class DataFrameSuite extends QueryTest // Assert that no extra shuffle introduced by cogroup. val exchanges = collect(df3.queryExecution.executedPlan) { @@ -298,7 +309,7 @@ index 002719f0689..fa76c42feaa 100644 } assert(exchanges.size == 2) } -@@ -3364,7 +3364,8 @@ class DataFrameSuite extends QueryTest +@@ -3364,7 +3367,8 @@ class DataFrameSuite extends QueryTest assert(df2.isLocal) } @@ -331,7 +342,7 @@ index c2fe31520ac..0f54b233d14 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..10702dba7a9 100644 +index f33432ddb6f..8690c859d0c 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 @@ -382,7 +393,17 @@ index f33432ddb6f..10702dba7a9 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { withTable("fact", "dim") { spark.range(100).select( -@@ -1187,7 +1194,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1027,7 +1034,8 @@ abstract class DynamicPartitionPruningSuiteBase + } + } + +- test("avoid reordering broadcast join keys to match input hash partitioning") { ++ test("avoid reordering broadcast join keys to match input hash partitioning", ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + withTable("large", "dimTwo", "dimThree") { +@@ -1187,7 +1195,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -392,7 +413,7 @@ index f33432ddb6f..10702dba7a9 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( """ -@@ -1238,7 +1246,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1238,7 +1247,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -402,7 +423,7 @@ index f33432ddb6f..10702dba7a9 100644 Given("dynamic pruning filter on the build side") withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( -@@ -1279,7 +1288,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1279,7 +1289,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -412,7 +433,7 @@ index f33432ddb6f..10702dba7a9 100644 Seq(NO_CODEGEN, CODEGEN_ONLY).foreach { mode => Seq(true, false).foreach { pruning => withSQLConf( -@@ -1311,7 +1321,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1311,7 +1322,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -422,7 +443,17 @@ index f33432ddb6f..10702dba7a9 100644 withSQLConf( SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true", -@@ -1470,7 +1481,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1423,7 +1435,8 @@ abstract class DynamicPartitionPruningSuiteBase + } + } + +- test("SPARK-34637: DPP side broadcast query stage is created firstly") { ++ test("SPARK-34637: DPP side broadcast query stage is created firstly", ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { + val df = sql( + """ WITH v as ( +@@ -1470,7 +1483,8 @@ abstract class DynamicPartitionPruningSuiteBase checkAnswer(df, Row(3, 2) :: Row(3, 2) :: Row(3, 2) :: Row(3, 2) :: Nil) } @@ -432,7 +463,7 @@ index f33432ddb6f..10702dba7a9 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { val df = sql( """ -@@ -1485,7 +1497,7 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1485,7 +1499,7 @@ abstract class DynamicPartitionPruningSuiteBase } test("SPARK-38148: Do not add dynamic partition pruning if there exists static partition " + @@ -441,7 +472,7 @@ index f33432ddb6f..10702dba7a9 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { Seq( "f.store_id = 1" -> false, -@@ -1557,7 +1569,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1557,7 +1571,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -451,7 +482,7 @@ index f33432ddb6f..10702dba7a9 100644 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 +1601,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1588,7 +1603,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -461,7 +492,7 @@ index f33432ddb6f..10702dba7a9 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { val df = sql( """ -@@ -1617,7 +1631,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1617,7 +1633,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -471,7 +502,7 @@ index f33432ddb6f..10702dba7a9 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { val df = sql( """ -@@ -1729,6 +1744,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1729,6 +1746,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"))) @@ -627,17 +658,19 @@ index 7af826583bd..3c3def1eb67 100644 assert(shuffleMergeJoins.size == 1) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala -index 9dcf7ec2904..39b57fb166a 100644 +index 9dcf7ec2904..94a171d1aad 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala -@@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation +@@ -30,7 +30,8 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.expressions.{Ascending, GenericRow, SortOrder} import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} import org.apache.spark.sql.catalyst.plans.logical.{Filter, HintInfo, Join, JoinHint, NO_BROADCAST_AND_REPLICATION} +-import org.apache.spark.sql.execution.{BinaryExecNode, FilterExec, ProjectExec, SortExec, SparkPlan, WholeStageCodegenExec} +import org.apache.spark.sql.comet._ - import org.apache.spark.sql.execution.{BinaryExecNode, FilterExec, ProjectExec, SortExec, SparkPlan, WholeStageCodegenExec} ++import org.apache.spark.sql.execution.{BinaryExecNode, ColumnarToRowExec, FilterExec, InputAdapter, ProjectExec, SortExec, SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.exchange.{ShuffleExchangeExec, ShuffleExchangeLike} + import org.apache.spark.sql.execution.joins._ @@ -801,7 +802,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan } } @@ -648,7 +681,20 @@ index 9dcf7ec2904..39b57fb166a 100644 withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1", SQLConf.SORT_MERGE_JOIN_EXEC_BUFFER_IN_MEMORY_THRESHOLD.key -> "0", SQLConf.SORT_MERGE_JOIN_EXEC_BUFFER_SPILL_THRESHOLD.key -> "1") { -@@ -1176,9 +1178,11 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -927,10 +929,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + val physical = df.queryExecution.sparkPlan + val physicalJoins = physical.collect { + case j: SortMergeJoinExec => j ++ case j: CometSortMergeJoinExec => j.originalPlan.asInstanceOf[SortMergeJoinExec] + } + val executed = df.queryExecution.executedPlan + val executedJoins = collect(executed) { + case j: SortMergeJoinExec => j ++ case j: CometSortMergeJoinExec => j.originalPlan.asInstanceOf[SortMergeJoinExec] + } + // This only applies to the above tested queries, in which a child SortMergeJoin always + // contains the SortOrder required by its parent SortMergeJoin. Thus, SortExec should never +@@ -1176,9 +1180,11 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan val plan = df1.join(df2.hint("SHUFFLE_HASH"), $"k1" === $"k2", joinType) .groupBy($"k1").count() .queryExecution.executedPlan @@ -662,7 +708,7 @@ index 9dcf7ec2904..39b57fb166a 100644 }) } -@@ -1195,10 +1199,11 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1195,10 +1201,11 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan .join(df4.hint("SHUFFLE_MERGE"), $"k1" === $"k4", joinType) .queryExecution .executedPlan @@ -676,7 +722,7 @@ index 9dcf7ec2904..39b57fb166a 100644 }) // Test shuffled hash join -@@ -1208,10 +1213,13 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1208,10 +1215,13 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan .join(df4.hint("SHUFFLE_MERGE"), $"k1" === $"k4", joinType) .queryExecution .executedPlan @@ -693,7 +739,7 @@ index 9dcf7ec2904..39b57fb166a 100644 }) } -@@ -1302,12 +1310,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1302,12 +1312,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan inputDFs.foreach { case (df1, df2, joinExprs) => val smjDF = df1.join(df2.hint("SHUFFLE_MERGE"), joinExprs, "full") assert(collect(smjDF.queryExecution.executedPlan) { @@ -708,7 +754,58 @@ index 9dcf7ec2904..39b57fb166a 100644 // Same result between shuffled hash join and sort merge join checkAnswer(shjDF, smjResult) } -@@ -1485,7 +1493,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1366,12 +1376,14 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + val smjDF = df1.hint("SHUFFLE_MERGE").join(df2, joinExprs, "leftouter") + assert(collect(smjDF.queryExecution.executedPlan) { + case _: SortMergeJoinExec => true ++ case _: CometSortMergeJoinExec => true + }.size === 1) + val smjResult = smjDF.collect() + + val shjDF = df1.hint("SHUFFLE_HASH").join(df2, joinExprs, "leftouter") + assert(collect(shjDF.queryExecution.executedPlan) { + case _: ShuffledHashJoinExec => true ++ case _: CometHashJoinExec => true + }.size === 1) + // Same result between shuffled hash join and sort merge join + checkAnswer(shjDF, smjResult) +@@ -1382,12 +1394,14 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + val smjDF = df2.join(df1.hint("SHUFFLE_MERGE"), joinExprs, "rightouter") + assert(collect(smjDF.queryExecution.executedPlan) { + case _: SortMergeJoinExec => true ++ case _: CometSortMergeJoinExec => true + }.size === 1) + val smjResult = smjDF.collect() + + val shjDF = df2.join(df1.hint("SHUFFLE_HASH"), joinExprs, "rightouter") + assert(collect(shjDF.queryExecution.executedPlan) { + case _: ShuffledHashJoinExec => true ++ case _: CometHashJoinExec => true + }.size === 1) + // Same result between shuffled hash join and sort merge join + checkAnswer(shjDF, smjResult) +@@ -1431,13 +1445,19 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + assert(shjCodegenDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_ : ShuffledHashJoinExec) => true + case WholeStageCodegenExec(ProjectExec(_, _ : ShuffledHashJoinExec)) => true ++ case WholeStageCodegenExec(ColumnarToRowExec(InputAdapter(_: CometHashJoinExec))) => ++ true ++ case WholeStageCodegenExec(ColumnarToRowExec( ++ InputAdapter(CometProjectExec(_, _, _, _, _: CometHashJoinExec, _)))) => true + }.size === 1) + checkAnswer(shjCodegenDF, Seq.empty) + + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { + val shjNonCodegenDF = df1.join(df2.hint("SHUFFLE_HASH"), $"k1" === $"k2", joinType) + assert(shjNonCodegenDF.queryExecution.executedPlan.collect { +- case _: ShuffledHashJoinExec => true }.size === 1) ++ case _: ShuffledHashJoinExec => true ++ case _: CometHashJoinExec => true ++ }.size === 1) + checkAnswer(shjNonCodegenDF, Seq.empty) + } + } +@@ -1485,7 +1505,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan val plan = sql(getAggQuery(selectExpr, joinType)).queryExecution.executedPlan assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) // Have shuffle before aggregation @@ -718,7 +815,7 @@ index 9dcf7ec2904..39b57fb166a 100644 } def getJoinQuery(selectExpr: String, joinType: String): String = { -@@ -1514,9 +1523,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1514,9 +1535,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan } val plan = sql(getJoinQuery(selectExpr, joinType)).queryExecution.executedPlan assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) @@ -733,7 +830,7 @@ index 9dcf7ec2904..39b57fb166a 100644 } // Test output ordering is not preserved -@@ -1525,9 +1537,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1525,9 +1549,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan val selectExpr = "/*+ BROADCAST(left_t) */ k1 as k0" val plan = sql(getJoinQuery(selectExpr, joinType)).queryExecution.executedPlan assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) @@ -748,7 +845,7 @@ index 9dcf7ec2904..39b57fb166a 100644 } // Test singe partition -@@ -1537,7 +1552,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1537,7 +1564,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan |FROM range(0, 10, 1, 1) t1 FULL OUTER JOIN range(0, 10, 1, 1) t2 |""".stripMargin) val plan = fullJoinDF.queryExecution.executedPlan @@ -758,6 +855,28 @@ index 9dcf7ec2904..39b57fb166a 100644 checkAnswer(fullJoinDF, Row(100)) } } +@@ -1582,6 +1610,9 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + Seq(semiJoinDF, antiJoinDF).foreach { df => + assert(collect(df.queryExecution.executedPlan) { + case j: ShuffledHashJoinExec if j.ignoreDuplicatedKey == ignoreDuplicatedKey => true ++ case j: CometHashJoinExec ++ if j.originalPlan.asInstanceOf[ShuffledHashJoinExec].ignoreDuplicatedKey == ++ ignoreDuplicatedKey => true + }.size == 1) + } + } +@@ -1633,7 +1664,10 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + + test("SPARK-43113: Full outer join with duplicate stream-side references in condition (SHJ)") { + def check(plan: SparkPlan): Unit = { +- assert(collect(plan) { case _: ShuffledHashJoinExec => true }.size === 1) ++ assert(collect(plan) { ++ case _: ShuffledHashJoinExec => true ++ case _: CometHashJoinExec => true ++ }.size === 1) + } + dupStreamSideColTest("SHUFFLE_HASH", check) + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala index b5b34922694..a72403780c4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala @@ -950,6 +1069,29 @@ index 6b07c77aefb..8277661560e 100644 }) } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/WriteDistributionAndOrderingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/WriteDistributionAndOrderingSuite.scala +index 40938eb6424..fad0fc1e1f0 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/connector/WriteDistributionAndOrderingSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/WriteDistributionAndOrderingSuite.scala +@@ -21,7 +21,7 @@ package org.apache.spark.sql.connector + import java.sql.Date + import java.util.Collections + +-import org.apache.spark.sql.{catalyst, AnalysisException, DataFrame, Row} ++import org.apache.spark.sql.{catalyst, AnalysisException, DataFrame, IgnoreCometSuite, Row} + import org.apache.spark.sql.catalyst.expressions.{ApplyFunctionExpression, Cast, Literal} + import org.apache.spark.sql.catalyst.expressions.objects.Invoke + import org.apache.spark.sql.catalyst.plans.physical +@@ -45,7 +45,8 @@ import org.apache.spark.sql.util.QueryExecutionListener + import org.apache.spark.tags.SlowSQLTest + + @SlowSQLTest +-class WriteDistributionAndOrderingSuite extends DistributionAndOrderingSuiteBase { ++class WriteDistributionAndOrderingSuite extends DistributionAndOrderingSuiteBase ++ with IgnoreCometSuite { + import testImplicits._ + + before { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala index ae1c0a86a14..1d3b914fd64 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala @@ -1183,7 +1325,7 @@ index 5a413c77754..37eee6e2c07 100644 val df = spark.read.parquet(path).selectExpr(projection: _*) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala -index 68bae34790a..ea906fd1adc 100644 +index 68bae34790a..0cc77ad09d7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -26,9 +26,11 @@ import org.scalatest.time.SpanSugar._ @@ -1207,7 +1349,7 @@ index 68bae34790a..ea906fd1adc 100644 } } -@@ -124,30 +127,38 @@ class AdaptiveQueryExecSuite +@@ -124,30 +127,39 @@ class AdaptiveQueryExecSuite private def findTopLevelSortMergeJoin(plan: SparkPlan): Seq[SortMergeJoinExec] = { collect(plan) { case j: SortMergeJoinExec => j @@ -1229,6 +1371,7 @@ index 68bae34790a..ea906fd1adc 100644 case j: BaseJoinExec => j + case c: CometHashJoinExec => c.originalPlan.asInstanceOf[BaseJoinExec] + case c: CometSortMergeJoinExec => c.originalPlan.asInstanceOf[BaseJoinExec] ++ case c: CometBroadcastHashJoinExec => c.originalPlan.asInstanceOf[BaseJoinExec] } } @@ -1246,7 +1389,7 @@ index 68bae34790a..ea906fd1adc 100644 } } -@@ -191,6 +202,7 @@ class AdaptiveQueryExecSuite +@@ -191,6 +203,7 @@ class AdaptiveQueryExecSuite val parts = rdd.partitions assert(parts.forall(rdd.preferredLocations(_).nonEmpty)) } @@ -1254,7 +1397,7 @@ index 68bae34790a..ea906fd1adc 100644 assert(numShuffles === (numLocalReads.length + numShufflesWithoutLocalRead)) } -@@ -199,7 +211,7 @@ class AdaptiveQueryExecSuite +@@ -199,7 +212,7 @@ class AdaptiveQueryExecSuite val plan = df.queryExecution.executedPlan assert(plan.isInstanceOf[AdaptiveSparkPlanExec]) val shuffle = plan.asInstanceOf[AdaptiveSparkPlanExec].executedPlan.collect { @@ -1263,7 +1406,7 @@ index 68bae34790a..ea906fd1adc 100644 } assert(shuffle.size == 1) assert(shuffle(0).outputPartitioning.numPartitions == numPartition) -@@ -215,7 +227,8 @@ class AdaptiveQueryExecSuite +@@ -215,7 +228,8 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) @@ -1273,7 +1416,7 @@ index 68bae34790a..ea906fd1adc 100644 } } -@@ -242,7 +255,8 @@ class AdaptiveQueryExecSuite +@@ -242,7 +256,8 @@ class AdaptiveQueryExecSuite } } @@ -1283,7 +1426,7 @@ index 68bae34790a..ea906fd1adc 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80", -@@ -274,7 +288,8 @@ class AdaptiveQueryExecSuite +@@ -274,7 +289,8 @@ class AdaptiveQueryExecSuite } } @@ -1293,7 +1436,7 @@ index 68bae34790a..ea906fd1adc 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80", -@@ -288,7 +303,8 @@ class AdaptiveQueryExecSuite +@@ -288,7 +304,8 @@ class AdaptiveQueryExecSuite val localReads = collect(adaptivePlan) { case read: AQEShuffleReadExec if read.isLocalRead => read } @@ -1303,7 +1446,29 @@ index 68bae34790a..ea906fd1adc 100644 val localShuffleRDD0 = localReads(0).execute().asInstanceOf[ShuffledRowRDD] val localShuffleRDD1 = localReads(1).execute().asInstanceOf[ShuffledRowRDD] // the final parallelism is math.max(1, numReduces / numMappers): math.max(1, 5/2) = 2 -@@ -337,7 +353,7 @@ class AdaptiveQueryExecSuite +@@ -313,7 +330,9 @@ class AdaptiveQueryExecSuite + .groupBy($"a").count() + checkAnswer(testDf, Seq()) + val plan = testDf.queryExecution.executedPlan +- assert(find(plan)(_.isInstanceOf[SortMergeJoinExec]).isDefined) ++ assert(find(plan) { case p => ++ p.isInstanceOf[SortMergeJoinExec] || p.isInstanceOf[CometSortMergeJoinExec] ++ }.isDefined) + val coalescedReads = collect(plan) { + case r: AQEShuffleReadExec => r + } +@@ -327,7 +346,9 @@ class AdaptiveQueryExecSuite + .groupBy($"a").count() + checkAnswer(testDf, Seq()) + val plan = testDf.queryExecution.executedPlan +- assert(find(plan)(_.isInstanceOf[BroadcastHashJoinExec]).isDefined) ++ assert(find(plan) { case p => ++ p.isInstanceOf[BroadcastHashJoinExec] || p.isInstanceOf[CometBroadcastHashJoinExec] ++ }.isDefined) + val coalescedReads = collect(plan) { + case r: AQEShuffleReadExec => r + } +@@ -337,7 +358,7 @@ class AdaptiveQueryExecSuite } } @@ -1312,7 +1477,7 @@ index 68bae34790a..ea906fd1adc 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { -@@ -352,7 +368,7 @@ class AdaptiveQueryExecSuite +@@ -352,7 +373,7 @@ class AdaptiveQueryExecSuite } } @@ -1321,7 +1486,7 @@ index 68bae34790a..ea906fd1adc 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { -@@ -368,7 +384,7 @@ class AdaptiveQueryExecSuite +@@ -368,7 +389,7 @@ class AdaptiveQueryExecSuite } } @@ -1330,7 +1495,7 @@ index 68bae34790a..ea906fd1adc 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { -@@ -413,7 +429,7 @@ class AdaptiveQueryExecSuite +@@ -413,7 +434,7 @@ class AdaptiveQueryExecSuite } } @@ -1339,7 +1504,7 @@ index 68bae34790a..ea906fd1adc 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { -@@ -458,7 +474,7 @@ class AdaptiveQueryExecSuite +@@ -458,7 +479,7 @@ class AdaptiveQueryExecSuite } } @@ -1348,7 +1513,16 @@ index 68bae34790a..ea906fd1adc 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "500") { -@@ -523,7 +539,7 @@ class AdaptiveQueryExecSuite +@@ -504,7 +525,7 @@ class AdaptiveQueryExecSuite + } + } + +- test("Exchange reuse") { ++ test("Exchange reuse", IgnoreComet("Comet shuffle changes shuffle metrics")) { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { +@@ -523,7 +544,7 @@ class AdaptiveQueryExecSuite } } @@ -1357,7 +1531,7 @@ index 68bae34790a..ea906fd1adc 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { -@@ -554,7 +570,9 @@ class AdaptiveQueryExecSuite +@@ -554,7 +575,9 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) @@ -1368,7 +1542,7 @@ index 68bae34790a..ea906fd1adc 100644 // Even with local shuffle read, the query stage reuse can also work. val ex = findReusedExchange(adaptivePlan) assert(ex.nonEmpty) -@@ -575,7 +593,9 @@ class AdaptiveQueryExecSuite +@@ -575,7 +598,9 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) @@ -1379,7 +1553,7 @@ index 68bae34790a..ea906fd1adc 100644 // Even with local shuffle read, the query stage reuse can also work. val ex = findReusedExchange(adaptivePlan) assert(ex.isEmpty) -@@ -584,7 +604,8 @@ class AdaptiveQueryExecSuite +@@ -584,7 +609,8 @@ class AdaptiveQueryExecSuite } } @@ -1389,7 +1563,7 @@ index 68bae34790a..ea906fd1adc 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "20000000", -@@ -679,7 +700,8 @@ class AdaptiveQueryExecSuite +@@ -679,7 +705,8 @@ class AdaptiveQueryExecSuite val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) // There is still a SMJ, and its two shuffles can't apply local read. @@ -1399,7 +1573,7 @@ index 68bae34790a..ea906fd1adc 100644 } } -@@ -801,7 +823,8 @@ class AdaptiveQueryExecSuite +@@ -801,7 +828,8 @@ class AdaptiveQueryExecSuite } } @@ -1409,7 +1583,7 @@ index 68bae34790a..ea906fd1adc 100644 Seq("SHUFFLE_MERGE", "SHUFFLE_HASH").foreach { joinHint => def getJoinNode(plan: SparkPlan): Seq[ShuffledJoin] = if (joinHint == "SHUFFLE_MERGE") { findTopLevelSortMergeJoin(plan) -@@ -1019,7 +1042,8 @@ class AdaptiveQueryExecSuite +@@ -1019,7 +1047,8 @@ class AdaptiveQueryExecSuite } } @@ -1419,7 +1593,7 @@ index 68bae34790a..ea906fd1adc 100644 withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { val (_, adaptivePlan) = runAdaptiveAndVerifyResult( "SELECT key FROM testData GROUP BY key") -@@ -1614,7 +1638,7 @@ class AdaptiveQueryExecSuite +@@ -1614,7 +1643,7 @@ class AdaptiveQueryExecSuite val (_, adaptivePlan) = runAdaptiveAndVerifyResult( "SELECT id FROM v1 GROUP BY id DISTRIBUTE BY id") assert(collect(adaptivePlan) { @@ -1428,7 +1602,7 @@ index 68bae34790a..ea906fd1adc 100644 }.length == 1) } } -@@ -1694,7 +1718,8 @@ class AdaptiveQueryExecSuite +@@ -1694,7 +1723,8 @@ class AdaptiveQueryExecSuite } } @@ -1438,7 +1612,7 @@ index 68bae34790a..ea906fd1adc 100644 def hasRepartitionShuffle(plan: SparkPlan): Boolean = { find(plan) { case s: ShuffleExchangeLike => -@@ -1879,6 +1904,9 @@ class AdaptiveQueryExecSuite +@@ -1879,6 +1909,9 @@ class AdaptiveQueryExecSuite def checkNoCoalescePartitions(ds: Dataset[Row], origin: ShuffleOrigin): Unit = { assert(collect(ds.queryExecution.executedPlan) { case s: ShuffleExchangeExec if s.shuffleOrigin == origin && s.numPartitions == 2 => s @@ -1448,7 +1622,7 @@ index 68bae34790a..ea906fd1adc 100644 }.size == 1) ds.collect() val plan = ds.queryExecution.executedPlan -@@ -1887,6 +1915,9 @@ class AdaptiveQueryExecSuite +@@ -1887,6 +1920,9 @@ class AdaptiveQueryExecSuite }.isEmpty) assert(collect(plan) { case s: ShuffleExchangeExec if s.shuffleOrigin == origin && s.numPartitions == 2 => s @@ -1458,7 +1632,7 @@ index 68bae34790a..ea906fd1adc 100644 }.size == 1) checkAnswer(ds, testData) } -@@ -2043,7 +2074,8 @@ class AdaptiveQueryExecSuite +@@ -2043,7 +2079,8 @@ class AdaptiveQueryExecSuite } } @@ -1468,7 +1642,7 @@ index 68bae34790a..ea906fd1adc 100644 withTempView("t1", "t2") { def checkJoinStrategy(shouldShuffleHashJoin: Boolean): Unit = { Seq("100", "100000").foreach { size => -@@ -2129,7 +2161,8 @@ class AdaptiveQueryExecSuite +@@ -2129,7 +2166,8 @@ class AdaptiveQueryExecSuite } } @@ -1478,7 +1652,7 @@ index 68bae34790a..ea906fd1adc 100644 withTempView("v") { withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", -@@ -2228,7 +2261,7 @@ class AdaptiveQueryExecSuite +@@ -2228,7 +2266,7 @@ class AdaptiveQueryExecSuite runAdaptiveAndVerifyResult(s"SELECT $repartition key1 FROM skewData1 " + s"JOIN skewData2 ON key1 = key2 GROUP BY key1") val shuffles1 = collect(adaptive1) { @@ -1487,7 +1661,7 @@ index 68bae34790a..ea906fd1adc 100644 } assert(shuffles1.size == 3) // shuffles1.head is the top-level shuffle under the Aggregate operator -@@ -2241,7 +2274,7 @@ class AdaptiveQueryExecSuite +@@ -2241,7 +2279,7 @@ class AdaptiveQueryExecSuite runAdaptiveAndVerifyResult(s"SELECT $repartition key1 FROM skewData1 " + s"JOIN skewData2 ON key1 = key2") val shuffles2 = collect(adaptive2) { @@ -1496,7 +1670,7 @@ index 68bae34790a..ea906fd1adc 100644 } if (hasRequiredDistribution) { assert(shuffles2.size == 3) -@@ -2275,7 +2308,8 @@ class AdaptiveQueryExecSuite +@@ -2275,7 +2313,8 @@ class AdaptiveQueryExecSuite } } @@ -1506,7 +1680,7 @@ index 68bae34790a..ea906fd1adc 100644 CostEvaluator.instantiate( classOf[SimpleShuffleSortCostEvaluator].getCanonicalName, spark.sparkContext.getConf) intercept[IllegalArgumentException] { -@@ -2419,6 +2453,7 @@ class AdaptiveQueryExecSuite +@@ -2419,6 +2458,7 @@ class AdaptiveQueryExecSuite val (_, adaptive) = runAdaptiveAndVerifyResult(query) assert(adaptive.collect { case sort: SortExec => sort @@ -1514,7 +1688,7 @@ index 68bae34790a..ea906fd1adc 100644 }.size == 1) val read = collect(adaptive) { case read: AQEShuffleReadExec => read -@@ -2436,7 +2471,8 @@ class AdaptiveQueryExecSuite +@@ -2436,7 +2476,8 @@ class AdaptiveQueryExecSuite } } @@ -1524,7 +1698,7 @@ index 68bae34790a..ea906fd1adc 100644 withTempView("v") { withSQLConf( SQLConf.ADAPTIVE_OPTIMIZE_SKEWS_IN_REBALANCE_PARTITIONS_ENABLED.key -> "true", -@@ -2548,7 +2584,7 @@ class AdaptiveQueryExecSuite +@@ -2548,7 +2589,7 @@ class AdaptiveQueryExecSuite runAdaptiveAndVerifyResult("SELECT key1 FROM skewData1 JOIN skewData2 ON key1 = key2 " + "JOIN skewData3 ON value2 = value3") val shuffles1 = collect(adaptive1) { @@ -1533,7 +1707,7 @@ index 68bae34790a..ea906fd1adc 100644 } assert(shuffles1.size == 4) val smj1 = findTopLevelSortMergeJoin(adaptive1) -@@ -2559,7 +2595,7 @@ class AdaptiveQueryExecSuite +@@ -2559,7 +2600,7 @@ class AdaptiveQueryExecSuite runAdaptiveAndVerifyResult("SELECT key1 FROM skewData1 JOIN skewData2 ON key1 = key2 " + "JOIN skewData3 ON value1 = value3") val shuffles2 = collect(adaptive2) { @@ -1542,6 +1716,14 @@ index 68bae34790a..ea906fd1adc 100644 } assert(shuffles2.size == 4) val smj2 = findTopLevelSortMergeJoin(adaptive2) +@@ -2756,6 +2797,7 @@ class AdaptiveQueryExecSuite + }.size == (if (firstAccess) 1 else 0)) + assert(collect(initialExecutedPlan) { + case s: SortExec => s ++ case s: CometSortExec => s + }.size == (if (firstAccess) 2 else 0)) + assert(collect(initialExecutedPlan) { + case i: InMemoryTableScanExec => i diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceCustomMetadataStructSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceCustomMetadataStructSuite.scala index 05872d41131..a2c328b9742 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceCustomMetadataStructSuite.scala @@ -1596,10 +1778,10 @@ index bf496d6db21..1e92016830f 100644 assert(fileSourceScanSchemata.size === expectedSchemaCatalogStrings.size, s"Found ${fileSourceScanSchemata.size} file sources in dataframe, " + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala -index ce43edb79c1..c414b19eda7 100644 +index ce43edb79c1..8436cb727c6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala -@@ -17,7 +17,7 @@ +@@ -17,9 +17,10 @@ package org.apache.spark.sql.execution.datasources @@ -1607,8 +1789,27 @@ index ce43edb79c1..c414b19eda7 100644 +import org.apache.spark.sql.{IgnoreComet, QueryTest, Row} import org.apache.spark.sql.catalyst.expressions.{Ascending, AttributeReference, NullsFirst, SortOrder} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Sort} ++import org.apache.spark.sql.comet.CometSortExec import org.apache.spark.sql.execution.{QueryExecution, SortExec} -@@ -305,7 +305,8 @@ class V1WriteCommandSuite extends QueryTest with SharedSparkSession with V1Write + import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec + import org.apache.spark.sql.internal.SQLConf +@@ -225,6 +226,7 @@ class V1WriteCommandSuite extends QueryTest with SharedSparkSession with V1Write + // assert the outer most sort in the executed plan + assert(plan.collectFirst { + case s: SortExec => s ++ case s: CometSortExec => s.originalPlan.asInstanceOf[SortExec] + }.exists { + case SortExec(Seq( + SortOrder(AttributeReference("key", IntegerType, _, _), Ascending, NullsFirst, _), +@@ -272,6 +274,7 @@ class V1WriteCommandSuite extends QueryTest with SharedSparkSession with V1Write + // assert the outer most sort in the executed plan + assert(plan.collectFirst { + case s: SortExec => s ++ case s: CometSortExec => s.originalPlan.asInstanceOf[SortExec] + }.exists { + case SortExec(Seq( + SortOrder(AttributeReference("value", StringType, _, _), Ascending, NullsFirst, _), +@@ -305,7 +308,8 @@ class V1WriteCommandSuite extends QueryTest with SharedSparkSession with V1Write } } @@ -2375,6 +2576,55 @@ index b4c4ec7acbf..20579284856 100644 } val aggregateExecsWithoutPartialAgg = allAggregateExecs.filter { +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala +index 3e1bc57dfa2..4a8d75ff512 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala +@@ -31,7 +31,7 @@ import org.apache.spark.scheduler.ExecutorCacheTaskLocation + import org.apache.spark.sql.{DataFrame, Row, SparkSession} + import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression} + import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning +-import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec ++import org.apache.spark.sql.execution.exchange.ShuffleExchangeLike + import org.apache.spark.sql.execution.streaming.{MemoryStream, StatefulOperatorStateInfo, StreamingSymmetricHashJoinExec, StreamingSymmetricHashJoinHelper} + import org.apache.spark.sql.execution.streaming.state.{RocksDBStateStoreProvider, StateStore, StateStoreProviderId} + import org.apache.spark.sql.functions._ +@@ -619,14 +619,27 @@ class StreamingInnerJoinSuite extends StreamingJoinSuite { + + val numPartitions = spark.sqlContext.conf.getConf(SQLConf.SHUFFLE_PARTITIONS) + +- assert(query.lastExecution.executedPlan.collect { +- case j @ StreamingSymmetricHashJoinExec(_, _, _, _, _, _, _, _, _, +- ShuffleExchangeExec(opA: HashPartitioning, _, _, _), +- ShuffleExchangeExec(opB: HashPartitioning, _, _, _)) +- if partitionExpressionsColumns(opA.expressions) === Seq("a", "b") +- && partitionExpressionsColumns(opB.expressions) === Seq("a", "b") +- && opA.numPartitions == numPartitions && opB.numPartitions == numPartitions => j +- }.size == 1) ++ val join = query.lastExecution.executedPlan.collect { ++ case j: StreamingSymmetricHashJoinExec => j ++ }.head ++ val opA = join.left.collect { ++ case s: ShuffleExchangeLike ++ if s.outputPartitioning.isInstanceOf[HashPartitioning] && ++ partitionExpressionsColumns( ++ s.outputPartitioning ++ .asInstanceOf[HashPartitioning].expressions) === Seq("a", "b") => ++ s.outputPartitioning.asInstanceOf[HashPartitioning] ++ }.head ++ val opB = join.right.collect { ++ case s: ShuffleExchangeLike ++ if s.outputPartitioning.isInstanceOf[HashPartitioning] && ++ partitionExpressionsColumns( ++ s.outputPartitioning ++ .asInstanceOf[HashPartitioning].expressions) === Seq("a", "b") => ++ s.outputPartitioning ++ .asInstanceOf[HashPartitioning] ++ }.head ++ assert(opA.numPartitions == numPartitions && opB.numPartitions == numPartitions) + }) + } + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala index abe606ad9c1..2d930b64cca 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala diff --git a/dev/diffs/4.0.0-preview1.diff b/dev/diffs/4.0.0-preview1.diff index f876f92ee..395ffab07 100644 --- a/dev/diffs/4.0.0-preview1.diff +++ b/dev/diffs/4.0.0-preview1.diff @@ -274,7 +274,7 @@ index 21a3ce1e122..316788b2989 100644 SET spark.sql.ansi.enabled = false; diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala -index d023fb82185..3774fa99148 100644 +index d023fb82185..0f4f03bda6c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeConstants @@ -296,6 +296,20 @@ index d023fb82185..3774fa99148 100644 } test("A cached table preserves the partitioning and ordering of its cached SparkPlan") { +@@ -1658,7 +1659,12 @@ class CachedTableSuite extends QueryTest with SQLTestUtils + _.nodeName.contains("TableCacheQueryStage")) + val aqeNode = findNodeInSparkPlanInfo(inMemoryScanNode.get, + _.nodeName.contains("AdaptiveSparkPlan")) +- aqeNode.get.children.head.nodeName == "AQEShuffleRead" ++ aqeNode.get.children.head.nodeName == "AQEShuffleRead" || ++ (aqeNode.get.children.head.nodeName.contains("WholeStageCodegen") && ++ aqeNode.get.children.head.children.head.nodeName == "ColumnarToRow" && ++ aqeNode.get.children.head.children.head.children.head.nodeName == "InputAdapter" && ++ aqeNode.get.children.head.children.head.children.head.children.head.nodeName == ++ "AQEShuffleRead") + } + + withTempView("t0", "t1", "t2") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index 620ee430cab..9d383a4bff9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -334,10 +348,15 @@ index f6fd6b501d7..11870c85d82 100644 spark.range(100).write.saveAsTable(s"$dbName.$table2Name") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala -index 760ee802608..db4dc90475e 100644 +index 760ee802608..ebd4a34b08d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala -@@ -40,7 +40,7 @@ import org.apache.spark.sql.connector.FakeV2Provider +@@ -36,11 +36,12 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, + import org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation + import org.apache.spark.sql.catalyst.parser.ParseException + import org.apache.spark.sql.catalyst.plans.logical.{Filter, LeafNode, LocalRelation, LogicalPlan, OneRowRelation} ++import org.apache.spark.sql.comet.CometBroadcastExchangeExec + import org.apache.spark.sql.connector.FakeV2Provider import org.apache.spark.sql.execution.{FilterExec, LogicalRDD, QueryExecution, SortExec, WholeStageCodegenExec} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.aggregate.HashAggregateExec @@ -346,7 +365,7 @@ index 760ee802608..db4dc90475e 100644 import org.apache.spark.sql.expressions.{Aggregator, Window} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -@@ -1401,7 +1401,7 @@ class DataFrameSuite extends QueryTest +@@ -1401,7 +1402,7 @@ class DataFrameSuite extends QueryTest fail("Should not have back to back Aggregates") } atFirstAgg = true @@ -355,7 +374,7 @@ index 760ee802608..db4dc90475e 100644 case _ => } } -@@ -1591,7 +1591,7 @@ class DataFrameSuite extends QueryTest +@@ -1591,7 +1592,7 @@ class DataFrameSuite extends QueryTest checkAnswer(join, df) assert( collect(join.queryExecution.executedPlan) { @@ -364,7 +383,7 @@ index 760ee802608..db4dc90475e 100644 assert( collect(join.queryExecution.executedPlan) { case e: ReusedExchangeExec => true }.size === 1) val broadcasted = broadcast(join) -@@ -1599,7 +1599,7 @@ class DataFrameSuite extends QueryTest +@@ -1599,10 +1600,12 @@ class DataFrameSuite extends QueryTest checkAnswer(join2, df) assert( collect(join2.queryExecution.executedPlan) { @@ -372,8 +391,14 @@ index 760ee802608..db4dc90475e 100644 + case _: ShuffleExchangeLike => true }.size == 1) assert( collect(join2.queryExecution.executedPlan) { - case e: BroadcastExchangeExec => true }.size === 1) -@@ -2000,7 +2000,7 @@ class DataFrameSuite extends QueryTest +- case e: BroadcastExchangeExec => true }.size === 1) ++ case e: BroadcastExchangeExec => true ++ case _: CometBroadcastExchangeExec => true ++ }.size === 1) + assert( + collect(join2.queryExecution.executedPlan) { case e: ReusedExchangeExec => true }.size == 4) + } +@@ -2000,7 +2003,7 @@ class DataFrameSuite extends QueryTest // Assert that no extra shuffle introduced by cogroup. val exchanges = collect(df3.queryExecution.executedPlan) { @@ -382,7 +407,7 @@ index 760ee802608..db4dc90475e 100644 } assert(exchanges.size == 2) } -@@ -2299,7 +2299,8 @@ class DataFrameSuite extends QueryTest +@@ -2299,7 +2302,8 @@ class DataFrameSuite extends QueryTest assert(df2.isLocal) } @@ -415,7 +440,7 @@ index 16a493b5290..3f0b70e2d59 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 2c24cc7d570..e7ea9b4a824 100644 +index 2c24cc7d570..de265cfaeae 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 @@ -466,7 +491,17 @@ index 2c24cc7d570..e7ea9b4a824 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { withTable("fact", "dim") { spark.range(100).select( -@@ -1187,7 +1194,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1027,7 +1034,8 @@ abstract class DynamicPartitionPruningSuiteBase + } + } + +- test("avoid reordering broadcast join keys to match input hash partitioning") { ++ test("avoid reordering broadcast join keys to match input hash partitioning", ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + withTable("large", "dimTwo", "dimThree") { +@@ -1187,7 +1195,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -476,7 +511,7 @@ index 2c24cc7d570..e7ea9b4a824 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( """ -@@ -1238,7 +1246,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1238,7 +1247,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -486,7 +521,7 @@ index 2c24cc7d570..e7ea9b4a824 100644 Given("dynamic pruning filter on the build side") withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( -@@ -1279,7 +1288,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1279,7 +1289,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -496,7 +531,7 @@ index 2c24cc7d570..e7ea9b4a824 100644 Seq(NO_CODEGEN, CODEGEN_ONLY).foreach { mode => Seq(true, false).foreach { pruning => withSQLConf( -@@ -1311,7 +1321,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1311,7 +1322,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -506,7 +541,17 @@ index 2c24cc7d570..e7ea9b4a824 100644 withSQLConf( SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true", -@@ -1471,7 +1482,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1424,7 +1436,8 @@ abstract class DynamicPartitionPruningSuiteBase + } + } + +- test("SPARK-34637: DPP side broadcast query stage is created firstly") { ++ test("SPARK-34637: DPP side broadcast query stage is created firstly", ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { + val df = sql( + """ WITH v as ( +@@ -1471,7 +1484,8 @@ abstract class DynamicPartitionPruningSuiteBase checkAnswer(df, Row(3, 2) :: Row(3, 2) :: Row(3, 2) :: Row(3, 2) :: Nil) } @@ -516,7 +561,7 @@ index 2c24cc7d570..e7ea9b4a824 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { val df = sql( """ -@@ -1486,7 +1498,7 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1486,7 +1500,7 @@ abstract class DynamicPartitionPruningSuiteBase } test("SPARK-38148: Do not add dynamic partition pruning if there exists static partition " + @@ -525,7 +570,7 @@ index 2c24cc7d570..e7ea9b4a824 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { Seq( "f.store_id = 1" -> false, -@@ -1558,7 +1570,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1558,7 +1572,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -535,7 +580,7 @@ index 2c24cc7d570..e7ea9b4a824 100644 withTable("duplicate_keys") { withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { Seq[(Int, String)]((1, "NL"), (1, "NL"), (3, "US"), (3, "US"), (3, "US")) -@@ -1589,7 +1602,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1589,7 +1604,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -545,7 +590,7 @@ index 2c24cc7d570..e7ea9b4a824 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { val df = sql( """ -@@ -1618,7 +1632,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1618,7 +1634,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -555,7 +600,7 @@ index 2c24cc7d570..e7ea9b4a824 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { val df = sql( """ -@@ -1730,6 +1745,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1730,6 +1747,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"))) @@ -711,17 +756,19 @@ index 53e47f428c3..a55d8f0c161 100644 assert(shuffleMergeJoins.size == 1) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala -index fcb937d82ba..f519436ba50 100644 +index fcb937d82ba..df79db88fed 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala -@@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation +@@ -29,7 +29,8 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.expressions.{Ascending, GenericRow, SortOrder} import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, JoinSelectionHelper} import org.apache.spark.sql.catalyst.plans.logical.{Filter, HintInfo, Join, JoinHint, NO_BROADCAST_AND_REPLICATION} +-import org.apache.spark.sql.execution.{BinaryExecNode, FilterExec, ProjectExec, SortExec, SparkPlan, WholeStageCodegenExec} +import org.apache.spark.sql.comet._ - import org.apache.spark.sql.execution.{BinaryExecNode, FilterExec, ProjectExec, SortExec, SparkPlan, WholeStageCodegenExec} ++import org.apache.spark.sql.execution.{BinaryExecNode, ColumnarToRowExec, FilterExec, InputAdapter, ProjectExec, SortExec, SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.exchange.{ShuffleExchangeExec, ShuffleExchangeLike} + import org.apache.spark.sql.execution.joins._ @@ -805,7 +806,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan } } @@ -732,7 +779,20 @@ index fcb937d82ba..f519436ba50 100644 withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1", SQLConf.SORT_MERGE_JOIN_EXEC_BUFFER_IN_MEMORY_THRESHOLD.key -> "0", SQLConf.SORT_MERGE_JOIN_EXEC_BUFFER_SPILL_THRESHOLD.key -> "1") { -@@ -1180,9 +1182,11 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -931,10 +933,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + val physical = df.queryExecution.sparkPlan + val physicalJoins = physical.collect { + case j: SortMergeJoinExec => j ++ case j: CometSortMergeJoinExec => j.originalPlan.asInstanceOf[SortMergeJoinExec] + } + val executed = df.queryExecution.executedPlan + val executedJoins = collect(executed) { + case j: SortMergeJoinExec => j ++ case j: CometSortMergeJoinExec => j.originalPlan.asInstanceOf[SortMergeJoinExec] + } + // This only applies to the above tested queries, in which a child SortMergeJoin always + // contains the SortOrder required by its parent SortMergeJoin. Thus, SortExec should never +@@ -1180,9 +1184,11 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan val plan = df1.join(df2.hint("SHUFFLE_HASH"), $"k1" === $"k2", joinType) .groupBy($"k1").count() .queryExecution.executedPlan @@ -746,7 +806,7 @@ index fcb937d82ba..f519436ba50 100644 }) } -@@ -1199,10 +1203,11 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1199,10 +1205,11 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan .join(df4.hint("SHUFFLE_MERGE"), $"k1" === $"k4", joinType) .queryExecution .executedPlan @@ -760,7 +820,7 @@ index fcb937d82ba..f519436ba50 100644 }) // Test shuffled hash join -@@ -1212,10 +1217,13 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1212,10 +1219,13 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan .join(df4.hint("SHUFFLE_MERGE"), $"k1" === $"k4", joinType) .queryExecution .executedPlan @@ -777,7 +837,7 @@ index fcb937d82ba..f519436ba50 100644 }) } -@@ -1306,12 +1314,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1306,12 +1316,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan inputDFs.foreach { case (df1, df2, joinExprs) => val smjDF = df1.join(df2.hint("SHUFFLE_MERGE"), joinExprs, "full") assert(collect(smjDF.queryExecution.executedPlan) { @@ -792,7 +852,58 @@ index fcb937d82ba..f519436ba50 100644 // Same result between shuffled hash join and sort merge join checkAnswer(shjDF, smjResult) } -@@ -1489,7 +1497,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1370,12 +1380,14 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + val smjDF = df1.hint("SHUFFLE_MERGE").join(df2, joinExprs, "leftouter") + assert(collect(smjDF.queryExecution.executedPlan) { + case _: SortMergeJoinExec => true ++ case _: CometSortMergeJoinExec => true + }.size === 1) + val smjResult = smjDF.collect() + + val shjDF = df1.hint("SHUFFLE_HASH").join(df2, joinExprs, "leftouter") + assert(collect(shjDF.queryExecution.executedPlan) { + case _: ShuffledHashJoinExec => true ++ case _: CometHashJoinExec => true + }.size === 1) + // Same result between shuffled hash join and sort merge join + checkAnswer(shjDF, smjResult) +@@ -1386,12 +1398,14 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + val smjDF = df2.join(df1.hint("SHUFFLE_MERGE"), joinExprs, "rightouter") + assert(collect(smjDF.queryExecution.executedPlan) { + case _: SortMergeJoinExec => true ++ case _: CometSortMergeJoinExec => true + }.size === 1) + val smjResult = smjDF.collect() + + val shjDF = df2.join(df1.hint("SHUFFLE_HASH"), joinExprs, "rightouter") + assert(collect(shjDF.queryExecution.executedPlan) { + case _: ShuffledHashJoinExec => true ++ case _: CometHashJoinExec => true + }.size === 1) + // Same result between shuffled hash join and sort merge join + checkAnswer(shjDF, smjResult) +@@ -1435,13 +1449,19 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + assert(shjCodegenDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_ : ShuffledHashJoinExec) => true + case WholeStageCodegenExec(ProjectExec(_, _ : ShuffledHashJoinExec)) => true ++ case WholeStageCodegenExec(ColumnarToRowExec(InputAdapter(_: CometHashJoinExec))) => ++ true ++ case WholeStageCodegenExec(ColumnarToRowExec( ++ InputAdapter(CometProjectExec(_, _, _, _, _: CometHashJoinExec, _)))) => true + }.size === 1) + checkAnswer(shjCodegenDF, Seq.empty) + + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { + val shjNonCodegenDF = df1.join(df2.hint("SHUFFLE_HASH"), $"k1" === $"k2", joinType) + assert(shjNonCodegenDF.queryExecution.executedPlan.collect { +- case _: ShuffledHashJoinExec => true }.size === 1) ++ case _: ShuffledHashJoinExec => true ++ case _: CometHashJoinExec => true ++ }.size === 1) + checkAnswer(shjNonCodegenDF, Seq.empty) + } + } +@@ -1489,7 +1509,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan val plan = sql(getAggQuery(selectExpr, joinType)).queryExecution.executedPlan assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) // Have shuffle before aggregation @@ -802,7 +913,7 @@ index fcb937d82ba..f519436ba50 100644 } def getJoinQuery(selectExpr: String, joinType: String): String = { -@@ -1518,9 +1527,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1518,9 +1539,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan } val plan = sql(getJoinQuery(selectExpr, joinType)).queryExecution.executedPlan assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) @@ -817,7 +928,7 @@ index fcb937d82ba..f519436ba50 100644 } // Test output ordering is not preserved -@@ -1529,9 +1541,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1529,9 +1553,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan val selectExpr = "/*+ BROADCAST(left_t) */ k1 as k0" val plan = sql(getJoinQuery(selectExpr, joinType)).queryExecution.executedPlan assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) @@ -832,7 +943,7 @@ index fcb937d82ba..f519436ba50 100644 } // Test singe partition -@@ -1541,7 +1556,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1541,7 +1568,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan |FROM range(0, 10, 1, 1) t1 FULL OUTER JOIN range(0, 10, 1, 1) t2 |""".stripMargin) val plan = fullJoinDF.queryExecution.executedPlan @@ -842,6 +953,38 @@ index fcb937d82ba..f519436ba50 100644 checkAnswer(fullJoinDF, Row(100)) } } +@@ -1586,6 +1614,9 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + Seq(semiJoinDF, antiJoinDF).foreach { df => + assert(collect(df.queryExecution.executedPlan) { + case j: ShuffledHashJoinExec if j.ignoreDuplicatedKey == ignoreDuplicatedKey => true ++ case j: CometHashJoinExec ++ if j.originalPlan.asInstanceOf[ShuffledHashJoinExec].ignoreDuplicatedKey == ++ ignoreDuplicatedKey => true + }.size == 1) + } + } +@@ -1637,7 +1668,10 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + + test("SPARK-43113: Full outer join with duplicate stream-side references in condition (SHJ)") { + def check(plan: SparkPlan): Unit = { +- assert(collect(plan) { case _: ShuffledHashJoinExec => true }.size === 1) ++ assert(collect(plan) { ++ case _: ShuffledHashJoinExec => true ++ case _: CometHashJoinExec => true ++ }.size === 1) + } + dupStreamSideColTest("SHUFFLE_HASH", check) + } +@@ -1773,7 +1807,8 @@ class ThreadLeakInSortMergeJoinSuite + sparkConf.set(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD, 20)) + } + +- test("SPARK-47146: thread leak when doing SortMergeJoin (with spill)") { ++ test("SPARK-47146: thread leak when doing SortMergeJoin (with spill)", ++ IgnoreComet("Comet SMJ doesn't spill yet")) { + + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala index 34c6c49bc49..f5dea07a213 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala @@ -1099,6 +1242,29 @@ index 10a32441b6c..5e5d763ee70 100644 }) } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/WriteDistributionAndOrderingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/WriteDistributionAndOrderingSuite.scala +index 12d5f13df01..816d1518c5b 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/connector/WriteDistributionAndOrderingSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/WriteDistributionAndOrderingSuite.scala +@@ -21,7 +21,7 @@ package org.apache.spark.sql.connector + import java.sql.Date + import java.util.Collections + +-import org.apache.spark.sql.{catalyst, AnalysisException, DataFrame, Row} ++import org.apache.spark.sql.{catalyst, AnalysisException, DataFrame, IgnoreCometSuite, Row} + import org.apache.spark.sql.catalyst.expressions.{ApplyFunctionExpression, Cast, Literal} + import org.apache.spark.sql.catalyst.expressions.objects.Invoke + import org.apache.spark.sql.catalyst.plans.physical +@@ -45,7 +45,8 @@ import org.apache.spark.sql.util.QueryExecutionListener + import org.apache.spark.tags.SlowSQLTest + + @SlowSQLTest +-class WriteDistributionAndOrderingSuite extends DistributionAndOrderingSuiteBase { ++class WriteDistributionAndOrderingSuite extends DistributionAndOrderingSuiteBase ++ with IgnoreCometSuite { + import testImplicits._ + + before { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala index 8238eabc7fe..c960fd75a9e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala @@ -1183,6 +1349,28 @@ index 15de4c5cc5b..6a85dfb6883 100644 import testImplicits._ setupTestData() +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala +index 3608e7c9207..6a05de2b9ac 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala +@@ -19,7 +19,7 @@ package org.apache.spark.sql.execution + import scala.collection.mutable + import scala.io.Source + +-import org.apache.spark.sql.{AnalysisException, Dataset, ExtendedExplainGenerator, FastOperator} ++import org.apache.spark.sql.{AnalysisException, Dataset, ExtendedExplainGenerator, FastOperator, IgnoreComet} + import org.apache.spark.sql.catalyst.{QueryPlanningTracker, QueryPlanningTrackerCallback} + import org.apache.spark.sql.catalyst.analysis.CurrentNamespace + import org.apache.spark.sql.catalyst.expressions.UnsafeRow +@@ -383,7 +383,7 @@ class QueryExecutionSuite extends SharedSparkSession { + } + } + +- test("SPARK-47289: extended explain info") { ++ test("SPARK-47289: extended explain info", IgnoreComet("Comet plan extended info is different")) { + val concat = new PlanStringConcat() + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", 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 b5bac8079c4..a3731888e12 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala @@ -1262,26 +1450,158 @@ index 47679ed7865..9ffbaecb98e 100644 assert(collectWithSubqueries(plan) { case s: SortAggregateExec => s }.length == sortAggCount) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala -index 3aaf61ffba4..93752e2a535 100644 +index 3aaf61ffba4..4130ece2283 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -22,6 +22,7 @@ import org.apache.spark.rdd.MapPartitionsWithEvaluatorRDD import org.apache.spark.sql.{Dataset, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode import org.apache.spark.sql.catalyst.expressions.codegen.{ByteCodeStats, CodeAndComment, CodeGenerator} -+import org.apache.spark.sql.comet.CometSortMergeJoinExec ++import org.apache.spark.sql.comet.{CometHashJoinExec, CometSortExec, CometSortMergeJoinExec} import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecutionSuite import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, SortAggregateExec} import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec -@@ -237,6 +238,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession +@@ -172,6 +173,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + val oneJoinDF = df1.join(df2.hint("SHUFFLE_HASH"), $"k1" === $"k2") + assert(oneJoinDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_ : ShuffledHashJoinExec) => true ++ case _: CometHashJoinExec => true + }.size === 1) + checkAnswer(oneJoinDF, Seq(Row(0, 0), Row(1, 1), Row(2, 2), Row(3, 3), Row(4, 4))) + +@@ -180,6 +182,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + .join(df3.hint("SHUFFLE_HASH"), $"k1" === $"k3") + assert(twoJoinsDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_ : ShuffledHashJoinExec) => true ++ case _: CometHashJoinExec => true + }.size === 2) + checkAnswer(twoJoinsDF, + Seq(Row(0, 0, 0), Row(1, 1, 1), Row(2, 2, 2), Row(3, 3, 3), Row(4, 4, 4))) +@@ -206,6 +209,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + assert(joinUniqueDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_ : ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true + case WholeStageCodegenExec(_ : SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true ++ case _: CometHashJoinExec if hint == "SHUFFLE_HASH" => true ++ case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true + }.size === 1) + checkAnswer(joinUniqueDF, Seq(Row(0, 0), Row(1, 1), Row(2, 2), Row(3, 3), Row(4, 4), + Row(null, 5), Row(null, 6), Row(null, 7), Row(null, 8), Row(null, 9))) +@@ -216,6 +221,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + assert(joinNonUniqueDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_ : ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true + case WholeStageCodegenExec(_ : SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true ++ case _: CometHashJoinExec if hint == "SHUFFLE_HASH" => true ++ case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true + }.size === 1) + checkAnswer(joinNonUniqueDF, Seq(Row(0, 0), Row(0, 3), Row(0, 6), Row(0, 9), Row(1, 1), + Row(1, 4), Row(1, 7), Row(2, 2), Row(2, 5), Row(2, 8), Row(3, null), Row(4, null))) +@@ -226,6 +233,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + assert(joinWithNonEquiDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_ : ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true + case WholeStageCodegenExec(_ : SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true ++ case _: CometHashJoinExec if hint == "SHUFFLE_HASH" => true ++ case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true + }.size === 1) + checkAnswer(joinWithNonEquiDF, Seq(Row(0, 0), Row(0, 6), Row(0, 9), Row(1, 1), + Row(1, 7), Row(2, 2), Row(2, 8), Row(3, null), Row(4, null), Row(null, 3), Row(null, 4), +@@ -237,6 +246,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession assert(twoJoinsDF.queryExecution.executedPlan.collect { case WholeStageCodegenExec(_ : ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true case WholeStageCodegenExec(_ : SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true ++ case _: CometHashJoinExec if hint == "SHUFFLE_HASH" => true + case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true }.size === 2) checkAnswer(twoJoinsDF, Seq(Row(0, 0, 0), Row(1, 1, null), Row(2, 2, 2), Row(3, 3, null), Row(4, 4, null), -@@ -360,6 +362,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession +@@ -258,6 +269,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + assert(rightJoinUniqueDf.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_: ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true + case WholeStageCodegenExec(_: SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true ++ case _: CometHashJoinExec if hint == "SHUFFLE_HASH" => true ++ case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true + }.size === 1) + checkAnswer(rightJoinUniqueDf, Seq(Row(1, 1), Row(2, 2), Row(3, 3), Row(4, 4), + Row(null, 5), Row(null, 6), Row(null, 7), Row(null, 8), Row(null, 9), +@@ -269,6 +282,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + assert(leftJoinUniqueDf.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_: ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true + case WholeStageCodegenExec(_: SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true ++ case _: CometHashJoinExec if hint == "SHUFFLE_HASH" => true ++ case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true + }.size === 1) + checkAnswer(leftJoinUniqueDf, Seq(Row(0, null), Row(1, 1), Row(2, 2), Row(3, 3), Row(4, 4))) + assert(leftJoinUniqueDf.count() === 5) +@@ -278,6 +293,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + assert(rightJoinNonUniqueDf.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_: ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true + case WholeStageCodegenExec(_: SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true ++ case _: CometHashJoinExec if hint == "SHUFFLE_HASH" => true ++ case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true + }.size === 1) + checkAnswer(rightJoinNonUniqueDf, Seq(Row(0, 3), Row(0, 6), Row(0, 9), Row(1, 1), + Row(1, 4), Row(1, 7), Row(1, 10), Row(2, 2), Row(2, 5), Row(2, 8))) +@@ -287,6 +304,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + assert(leftJoinNonUniqueDf.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_: ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true + case WholeStageCodegenExec(_: SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true ++ case _: CometHashJoinExec if hint == "SHUFFLE_HASH" => true ++ case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true + }.size === 1) + checkAnswer(leftJoinNonUniqueDf, Seq(Row(0, 3), Row(0, 6), Row(0, 9), Row(1, 1), + Row(1, 4), Row(1, 7), Row(1, 10), Row(2, 2), Row(2, 5), Row(2, 8), Row(3, null), +@@ -298,6 +317,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + assert(rightJoinWithNonEquiDf.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_: ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true + case WholeStageCodegenExec(_: SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true ++ case _: CometHashJoinExec if hint == "SHUFFLE_HASH" => true ++ case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true + }.size === 1) + checkAnswer(rightJoinWithNonEquiDf, Seq(Row(0, 6), Row(0, 9), Row(1, 1), Row(1, 7), + Row(1, 10), Row(2, 2), Row(2, 8), Row(null, 3), Row(null, 4), Row(null, 5))) +@@ -308,6 +329,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + assert(leftJoinWithNonEquiDf.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_: ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true + case WholeStageCodegenExec(_: SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true ++ case _: CometHashJoinExec if hint == "SHUFFLE_HASH" => true ++ case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true + }.size === 1) + checkAnswer(leftJoinWithNonEquiDf, Seq(Row(0, 6), Row(0, 9), Row(1, 1), Row(1, 7), + Row(1, 10), Row(2, 2), Row(2, 8), Row(3, null), Row(4, null))) +@@ -318,6 +341,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + assert(twoRightJoinsDf.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_: ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true + case WholeStageCodegenExec(_: SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true ++ case _: CometHashJoinExec if hint == "SHUFFLE_HASH" => true ++ case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true + }.size === 2) + checkAnswer(twoRightJoinsDf, Seq(Row(2, 2, 2), Row(3, 3, 3), Row(4, 4, 4))) + +@@ -327,6 +352,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + assert(twoLeftJoinsDf.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_: ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true + case WholeStageCodegenExec(_: SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true ++ case _: CometHashJoinExec if hint == "SHUFFLE_HASH" => true ++ case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true + }.size === 2) + checkAnswer(twoLeftJoinsDf, + Seq(Row(0, null, null), Row(1, 1, null), Row(2, 2, 2), Row(3, 3, 3), Row(4, 4, 4))) +@@ -343,6 +370,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + val oneLeftOuterJoinDF = df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", "left_outer") + assert(oneLeftOuterJoinDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_ : SortMergeJoinExec) => true ++ case _: CometSortMergeJoinExec => true + }.size === 1) + checkAnswer(oneLeftOuterJoinDF, Seq(Row(0, 0), Row(1, 1), Row(2, 2), Row(3, 3), Row(4, null), + Row(5, null), Row(6, null), Row(7, null), Row(8, null), Row(9, null))) +@@ -351,6 +379,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + val oneRightOuterJoinDF = df2.join(df3.hint("SHUFFLE_MERGE"), $"k2" === $"k3", "right_outer") + assert(oneRightOuterJoinDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_ : SortMergeJoinExec) => true ++ case _: CometSortMergeJoinExec => true + }.size === 1) + checkAnswer(oneRightOuterJoinDF, Seq(Row(0, 0), Row(1, 1), Row(2, 2), Row(3, 3), Row(null, 4), + Row(null, 5))) +@@ -360,6 +389,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession .join(df1.hint("SHUFFLE_MERGE"), $"k3" === $"k1", "right_outer") assert(twoJoinsDF.queryExecution.executedPlan.collect { case WholeStageCodegenExec(_ : SortMergeJoinExec) => true @@ -1289,38 +1609,57 @@ index 3aaf61ffba4..93752e2a535 100644 }.size === 2) checkAnswer(twoJoinsDF, Seq(Row(0, 0, 0), Row(1, 1, 1), Row(2, 2, 2), Row(3, 3, 3), Row(4, null, 4), Row(5, null, 5), -@@ -382,8 +385,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession +@@ -375,6 +405,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + val oneJoinDF = df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", "left_semi") + assert(oneJoinDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(ProjectExec(_, _ : SortMergeJoinExec)) => true ++ case _: CometSortMergeJoinExec => true + }.size === 1) + checkAnswer(oneJoinDF, Seq(Row(0), Row(1), Row(2), Row(3))) + +@@ -382,8 +413,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession val twoJoinsDF = df3.join(df2.hint("SHUFFLE_MERGE"), $"k3" === $"k2", "left_semi") .join(df1.hint("SHUFFLE_MERGE"), $"k3" === $"k1", "left_semi") assert(twoJoinsDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(ProjectExec(_, _ : SortMergeJoinExec)) | - WholeStageCodegenExec(_ : SortMergeJoinExec) => true + case _: SortMergeJoinExec => true ++ case _: CometSortMergeJoinExec => true }.size === 2) checkAnswer(twoJoinsDF, Seq(Row(0), Row(1), Row(2), Row(3))) } -@@ -404,8 +406,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession +@@ -397,6 +428,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + val oneJoinDF = df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", "left_anti") + assert(oneJoinDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(ProjectExec(_, _ : SortMergeJoinExec)) => true ++ case _: CometSortMergeJoinExec => true + }.size === 1) + checkAnswer(oneJoinDF, Seq(Row(4), Row(5), Row(6), Row(7), Row(8), Row(9))) + +@@ -404,8 +436,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession val twoJoinsDF = df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", "left_anti") .join(df3.hint("SHUFFLE_MERGE"), $"k1" === $"k3", "left_anti") assert(twoJoinsDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(ProjectExec(_, _ : SortMergeJoinExec)) | - WholeStageCodegenExec(_ : SortMergeJoinExec) => true + case _: SortMergeJoinExec => true ++ case _: CometSortMergeJoinExec => true }.size === 2) checkAnswer(twoJoinsDF, Seq(Row(6), Row(7), Row(8), Row(9))) } -@@ -538,7 +539,9 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession +@@ -538,7 +570,10 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession val plan = df.queryExecution.executedPlan assert(plan.exists(p => p.isInstanceOf[WholeStageCodegenExec] && - p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[SortExec])) + p.asInstanceOf[WholeStageCodegenExec].collect { + case _: SortExec => true ++ case _: CometSortExec => true + }.nonEmpty)) assert(df.collect() === Array(Row(1), Row(2), Row(3))) } -@@ -718,7 +721,9 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession +@@ -718,7 +753,9 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession .write.mode(SaveMode.Overwrite).parquet(path) withSQLConf(SQLConf.WHOLESTAGE_MAX_NUM_FIELDS.key -> "255", @@ -1332,7 +1671,7 @@ index 3aaf61ffba4..93752e2a535 100644 val df = spark.read.parquet(path).selectExpr(projection: _*) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala -index a7efd0aa75e..fa65bda2051 100644 +index a7efd0aa75e..baae0967a2a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -28,11 +28,13 @@ import org.apache.spark.SparkException @@ -1358,7 +1697,7 @@ index a7efd0aa75e..fa65bda2051 100644 } } -@@ -132,30 +135,38 @@ class AdaptiveQueryExecSuite +@@ -132,30 +135,39 @@ class AdaptiveQueryExecSuite private def findTopLevelSortMergeJoin(plan: SparkPlan): Seq[SortMergeJoinExec] = { collect(plan) { case j: SortMergeJoinExec => j @@ -1380,6 +1719,7 @@ index a7efd0aa75e..fa65bda2051 100644 case j: BaseJoinExec => j + case c: CometHashJoinExec => c.originalPlan.asInstanceOf[BaseJoinExec] + case c: CometSortMergeJoinExec => c.originalPlan.asInstanceOf[BaseJoinExec] ++ case c: CometBroadcastHashJoinExec => c.originalPlan.asInstanceOf[BaseJoinExec] } } @@ -1397,7 +1737,7 @@ index a7efd0aa75e..fa65bda2051 100644 } } -@@ -205,6 +216,7 @@ class AdaptiveQueryExecSuite +@@ -205,6 +217,7 @@ class AdaptiveQueryExecSuite val parts = rdd.partitions assert(parts.forall(rdd.preferredLocations(_).nonEmpty)) } @@ -1405,7 +1745,7 @@ index a7efd0aa75e..fa65bda2051 100644 assert(numShuffles === (numLocalReads.length + numShufflesWithoutLocalRead)) } -@@ -213,7 +225,7 @@ class AdaptiveQueryExecSuite +@@ -213,7 +226,7 @@ class AdaptiveQueryExecSuite val plan = df.queryExecution.executedPlan assert(plan.isInstanceOf[AdaptiveSparkPlanExec]) val shuffle = plan.asInstanceOf[AdaptiveSparkPlanExec].executedPlan.collect { @@ -1414,7 +1754,7 @@ index a7efd0aa75e..fa65bda2051 100644 } assert(shuffle.size == 1) assert(shuffle(0).outputPartitioning.numPartitions == numPartition) -@@ -229,7 +241,8 @@ class AdaptiveQueryExecSuite +@@ -229,7 +242,8 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) @@ -1424,7 +1764,7 @@ index a7efd0aa75e..fa65bda2051 100644 } } -@@ -256,7 +269,8 @@ class AdaptiveQueryExecSuite +@@ -256,7 +270,8 @@ class AdaptiveQueryExecSuite } } @@ -1434,7 +1774,7 @@ index a7efd0aa75e..fa65bda2051 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80", -@@ -288,7 +302,8 @@ class AdaptiveQueryExecSuite +@@ -288,7 +303,8 @@ class AdaptiveQueryExecSuite } } @@ -1444,7 +1784,7 @@ index a7efd0aa75e..fa65bda2051 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80", -@@ -302,7 +317,8 @@ class AdaptiveQueryExecSuite +@@ -302,7 +318,8 @@ class AdaptiveQueryExecSuite val localReads = collect(adaptivePlan) { case read: AQEShuffleReadExec if read.isLocalRead => read } @@ -1454,7 +1794,29 @@ index a7efd0aa75e..fa65bda2051 100644 val localShuffleRDD0 = localReads(0).execute().asInstanceOf[ShuffledRowRDD] val localShuffleRDD1 = localReads(1).execute().asInstanceOf[ShuffledRowRDD] // the final parallelism is math.max(1, numReduces / numMappers): math.max(1, 5/2) = 2 -@@ -351,7 +367,7 @@ class AdaptiveQueryExecSuite +@@ -327,7 +344,9 @@ class AdaptiveQueryExecSuite + .groupBy($"a").count() + checkAnswer(testDf, Seq()) + val plan = testDf.queryExecution.executedPlan +- assert(find(plan)(_.isInstanceOf[SortMergeJoinExec]).isDefined) ++ assert(find(plan) { case p => ++ p.isInstanceOf[SortMergeJoinExec] || p.isInstanceOf[CometSortMergeJoinExec] ++ }.isDefined) + val coalescedReads = collect(plan) { + case r: AQEShuffleReadExec => r + } +@@ -341,7 +360,9 @@ class AdaptiveQueryExecSuite + .groupBy($"a").count() + checkAnswer(testDf, Seq()) + val plan = testDf.queryExecution.executedPlan +- assert(find(plan)(_.isInstanceOf[BroadcastHashJoinExec]).isDefined) ++ assert(find(plan) { case p => ++ p.isInstanceOf[BroadcastHashJoinExec] || p.isInstanceOf[CometBroadcastHashJoinExec] ++ }.isDefined) + val coalescedReads = collect(plan) { + case r: AQEShuffleReadExec => r + } +@@ -351,7 +372,7 @@ class AdaptiveQueryExecSuite } } @@ -1463,7 +1825,7 @@ index a7efd0aa75e..fa65bda2051 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { -@@ -366,7 +382,7 @@ class AdaptiveQueryExecSuite +@@ -366,7 +387,7 @@ class AdaptiveQueryExecSuite } } @@ -1472,7 +1834,7 @@ index a7efd0aa75e..fa65bda2051 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { -@@ -382,7 +398,7 @@ class AdaptiveQueryExecSuite +@@ -382,7 +403,7 @@ class AdaptiveQueryExecSuite } } @@ -1481,7 +1843,7 @@ index a7efd0aa75e..fa65bda2051 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { -@@ -427,7 +443,7 @@ class AdaptiveQueryExecSuite +@@ -427,7 +448,7 @@ class AdaptiveQueryExecSuite } } @@ -1490,7 +1852,7 @@ index a7efd0aa75e..fa65bda2051 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { -@@ -472,7 +488,7 @@ class AdaptiveQueryExecSuite +@@ -472,7 +493,7 @@ class AdaptiveQueryExecSuite } } @@ -1499,7 +1861,16 @@ index a7efd0aa75e..fa65bda2051 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "500") { -@@ -537,7 +553,7 @@ class AdaptiveQueryExecSuite +@@ -518,7 +539,7 @@ class AdaptiveQueryExecSuite + } + } + +- test("Exchange reuse") { ++ test("Exchange reuse", IgnoreComet("Comet shuffle changes shuffle metrics")) { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { +@@ -537,7 +558,7 @@ class AdaptiveQueryExecSuite } } @@ -1508,7 +1879,7 @@ index a7efd0aa75e..fa65bda2051 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { -@@ -568,7 +584,9 @@ class AdaptiveQueryExecSuite +@@ -568,7 +589,9 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) @@ -1519,7 +1890,7 @@ index a7efd0aa75e..fa65bda2051 100644 // Even with local shuffle read, the query stage reuse can also work. val ex = findReusedExchange(adaptivePlan) assert(ex.nonEmpty) -@@ -589,7 +607,9 @@ class AdaptiveQueryExecSuite +@@ -589,7 +612,9 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) @@ -1530,7 +1901,7 @@ index a7efd0aa75e..fa65bda2051 100644 // Even with local shuffle read, the query stage reuse can also work. val ex = findReusedExchange(adaptivePlan) assert(ex.isEmpty) -@@ -598,7 +618,8 @@ class AdaptiveQueryExecSuite +@@ -598,7 +623,8 @@ class AdaptiveQueryExecSuite } } @@ -1540,7 +1911,7 @@ index a7efd0aa75e..fa65bda2051 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "20000000", -@@ -693,7 +714,8 @@ class AdaptiveQueryExecSuite +@@ -693,7 +719,8 @@ class AdaptiveQueryExecSuite val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) // There is still a SMJ, and its two shuffles can't apply local read. @@ -1550,7 +1921,7 @@ index a7efd0aa75e..fa65bda2051 100644 } } -@@ -815,7 +837,8 @@ class AdaptiveQueryExecSuite +@@ -815,7 +842,8 @@ class AdaptiveQueryExecSuite } } @@ -1560,7 +1931,7 @@ index a7efd0aa75e..fa65bda2051 100644 Seq("SHUFFLE_MERGE", "SHUFFLE_HASH").foreach { joinHint => def getJoinNode(plan: SparkPlan): Seq[ShuffledJoin] = if (joinHint == "SHUFFLE_MERGE") { findTopLevelSortMergeJoin(plan) -@@ -1123,7 +1146,8 @@ class AdaptiveQueryExecSuite +@@ -1123,7 +1151,8 @@ class AdaptiveQueryExecSuite } } @@ -1570,7 +1941,7 @@ index a7efd0aa75e..fa65bda2051 100644 withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { val (_, adaptivePlan) = runAdaptiveAndVerifyResult( "SELECT key FROM testData GROUP BY key") -@@ -1718,7 +1742,7 @@ class AdaptiveQueryExecSuite +@@ -1718,7 +1747,7 @@ class AdaptiveQueryExecSuite val (_, adaptivePlan) = runAdaptiveAndVerifyResult( "SELECT id FROM v1 GROUP BY id DISTRIBUTE BY id") assert(collect(adaptivePlan) { @@ -1579,7 +1950,7 @@ index a7efd0aa75e..fa65bda2051 100644 }.length == 1) } } -@@ -1798,7 +1822,8 @@ class AdaptiveQueryExecSuite +@@ -1798,7 +1827,8 @@ class AdaptiveQueryExecSuite } } @@ -1589,7 +1960,7 @@ index a7efd0aa75e..fa65bda2051 100644 def hasRepartitionShuffle(plan: SparkPlan): Boolean = { find(plan) { case s: ShuffleExchangeLike => -@@ -1983,6 +2008,9 @@ class AdaptiveQueryExecSuite +@@ -1983,6 +2013,9 @@ class AdaptiveQueryExecSuite def checkNoCoalescePartitions(ds: Dataset[Row], origin: ShuffleOrigin): Unit = { assert(collect(ds.queryExecution.executedPlan) { case s: ShuffleExchangeExec if s.shuffleOrigin == origin && s.numPartitions == 2 => s @@ -1599,7 +1970,7 @@ index a7efd0aa75e..fa65bda2051 100644 }.size == 1) ds.collect() val plan = ds.queryExecution.executedPlan -@@ -1991,6 +2019,9 @@ class AdaptiveQueryExecSuite +@@ -1991,6 +2024,9 @@ class AdaptiveQueryExecSuite }.isEmpty) assert(collect(plan) { case s: ShuffleExchangeExec if s.shuffleOrigin == origin && s.numPartitions == 2 => s @@ -1609,7 +1980,7 @@ index a7efd0aa75e..fa65bda2051 100644 }.size == 1) checkAnswer(ds, testData) } -@@ -2147,7 +2178,8 @@ class AdaptiveQueryExecSuite +@@ -2147,7 +2183,8 @@ class AdaptiveQueryExecSuite } } @@ -1619,7 +1990,7 @@ index a7efd0aa75e..fa65bda2051 100644 withTempView("t1", "t2") { def checkJoinStrategy(shouldShuffleHashJoin: Boolean): Unit = { Seq("100", "100000").foreach { size => -@@ -2233,7 +2265,8 @@ class AdaptiveQueryExecSuite +@@ -2233,7 +2270,8 @@ class AdaptiveQueryExecSuite } } @@ -1629,7 +2000,7 @@ index a7efd0aa75e..fa65bda2051 100644 withTempView("v") { withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", -@@ -2332,7 +2365,7 @@ class AdaptiveQueryExecSuite +@@ -2332,7 +2370,7 @@ class AdaptiveQueryExecSuite runAdaptiveAndVerifyResult(s"SELECT $repartition key1 FROM skewData1 " + s"JOIN skewData2 ON key1 = key2 GROUP BY key1") val shuffles1 = collect(adaptive1) { @@ -1638,7 +2009,7 @@ index a7efd0aa75e..fa65bda2051 100644 } assert(shuffles1.size == 3) // shuffles1.head is the top-level shuffle under the Aggregate operator -@@ -2345,7 +2378,7 @@ class AdaptiveQueryExecSuite +@@ -2345,7 +2383,7 @@ class AdaptiveQueryExecSuite runAdaptiveAndVerifyResult(s"SELECT $repartition key1 FROM skewData1 " + s"JOIN skewData2 ON key1 = key2") val shuffles2 = collect(adaptive2) { @@ -1647,7 +2018,7 @@ index a7efd0aa75e..fa65bda2051 100644 } if (hasRequiredDistribution) { assert(shuffles2.size == 3) -@@ -2379,7 +2412,8 @@ class AdaptiveQueryExecSuite +@@ -2379,7 +2417,8 @@ class AdaptiveQueryExecSuite } } @@ -1657,7 +2028,17 @@ index a7efd0aa75e..fa65bda2051 100644 CostEvaluator.instantiate( classOf[SimpleShuffleSortCostEvaluator].getCanonicalName, spark.sparkContext.getConf) intercept[IllegalArgumentException] { -@@ -2545,6 +2579,7 @@ class AdaptiveQueryExecSuite +@@ -2510,7 +2549,8 @@ class AdaptiveQueryExecSuite + } + + test("SPARK-48037: Fix SortShuffleWriter lacks shuffle write related metrics " + +- "resulting in potentially inaccurate data") { ++ "resulting in potentially inaccurate data", ++ IgnoreComet("too many shuffle partitions causes Java heap OOM")) { + withTable("t3") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", +@@ -2545,6 +2585,7 @@ class AdaptiveQueryExecSuite val (_, adaptive) = runAdaptiveAndVerifyResult(query) assert(adaptive.collect { case sort: SortExec => sort @@ -1665,7 +2046,7 @@ index a7efd0aa75e..fa65bda2051 100644 }.size == 1) val read = collect(adaptive) { case read: AQEShuffleReadExec => read -@@ -2562,7 +2597,8 @@ class AdaptiveQueryExecSuite +@@ -2562,7 +2603,8 @@ class AdaptiveQueryExecSuite } } @@ -1675,7 +2056,7 @@ index a7efd0aa75e..fa65bda2051 100644 withTempView("v") { withSQLConf( SQLConf.ADAPTIVE_OPTIMIZE_SKEWS_IN_REBALANCE_PARTITIONS_ENABLED.key -> "true", -@@ -2674,7 +2710,7 @@ class AdaptiveQueryExecSuite +@@ -2674,7 +2716,7 @@ class AdaptiveQueryExecSuite runAdaptiveAndVerifyResult("SELECT key1 FROM skewData1 JOIN skewData2 ON key1 = key2 " + "JOIN skewData3 ON value2 = value3") val shuffles1 = collect(adaptive1) { @@ -1684,7 +2065,7 @@ index a7efd0aa75e..fa65bda2051 100644 } assert(shuffles1.size == 4) val smj1 = findTopLevelSortMergeJoin(adaptive1) -@@ -2685,7 +2721,7 @@ class AdaptiveQueryExecSuite +@@ -2685,7 +2727,7 @@ class AdaptiveQueryExecSuite runAdaptiveAndVerifyResult("SELECT key1 FROM skewData1 JOIN skewData2 ON key1 = key2 " + "JOIN skewData3 ON value1 = value3") val shuffles2 = collect(adaptive2) { @@ -1693,6 +2074,22 @@ index a7efd0aa75e..fa65bda2051 100644 } assert(shuffles2.size == 4) val smj2 = findTopLevelSortMergeJoin(adaptive2) +@@ -2911,6 +2953,7 @@ class AdaptiveQueryExecSuite + }.size == (if (firstAccess) 1 else 0)) + assert(collect(initialExecutedPlan) { + case s: SortExec => s ++ case s: CometSortExec => s.originalPlan.asInstanceOf[SortExec] + }.size == (if (firstAccess) 2 else 0)) + assert(collect(initialExecutedPlan) { + case i: InMemoryTableScanLike => i +@@ -2923,6 +2966,7 @@ class AdaptiveQueryExecSuite + }.isEmpty) + assert(collect(finalExecutedPlan) { + case s: SortExec => s ++ case s: CometSortExec => s.originalPlan.asInstanceOf[SortExec] + }.isEmpty) + assert(collect(initialExecutedPlan) { + case i: InMemoryTableScanLike => i diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceCustomMetadataStructSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceCustomMetadataStructSuite.scala index 05872d41131..0dd83608bbd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceCustomMetadataStructSuite.scala @@ -1737,10 +2134,10 @@ index 0a0b23d1e60..5685926250f 100644 assert(fileSourceScanSchemata.size === expectedSchemaCatalogStrings.size, s"Found ${fileSourceScanSchemata.size} file sources in dataframe, " + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala -index 04a7b4834f4..3b0fd1eb5aa 100644 +index 04a7b4834f4..8cab62ce4ab 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala -@@ -17,7 +17,7 @@ +@@ -17,9 +17,10 @@ package org.apache.spark.sql.execution.datasources @@ -1748,8 +2145,27 @@ index 04a7b4834f4..3b0fd1eb5aa 100644 +import org.apache.spark.sql.{IgnoreComet, QueryTest, Row} import org.apache.spark.sql.catalyst.expressions.{Ascending, AttributeReference, NullsFirst, SortOrder} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Sort} ++import org.apache.spark.sql.comet.CometSortExec import org.apache.spark.sql.execution.{QueryExecution, SortExec} -@@ -305,7 +305,8 @@ class V1WriteCommandSuite extends QueryTest with SharedSparkSession with V1Write + import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec + import org.apache.spark.sql.internal.SQLConf +@@ -225,6 +226,7 @@ class V1WriteCommandSuite extends QueryTest with SharedSparkSession with V1Write + // assert the outer most sort in the executed plan + assert(plan.collectFirst { + case s: SortExec => s ++ case s: CometSortExec => s.originalPlan.asInstanceOf[SortExec] + }.exists { + case SortExec(Seq( + SortOrder(AttributeReference("key", IntegerType, _, _), Ascending, NullsFirst, _), +@@ -272,6 +274,7 @@ class V1WriteCommandSuite extends QueryTest with SharedSparkSession with V1Write + // assert the outer most sort in the executed plan + assert(plan.collectFirst { + case s: SortExec => s ++ case s: CometSortExec => s.originalPlan.asInstanceOf[SortExec] + }.exists { + case SortExec(Seq( + SortOrder(AttributeReference("value", StringType, _, _), Ascending, NullsFirst, _), +@@ -305,7 +308,8 @@ class V1WriteCommandSuite extends QueryTest with SharedSparkSession with V1Write } } @@ -2070,6 +2486,35 @@ index 4bd35e0789b..6544d86dbe0 100644 ) } test(s"parquet widening conversion $fromType -> $toType") { +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceReadSuite.scala +index c800168b507..991d52a1a75 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceReadSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceReadSuite.scala +@@ -22,7 +22,7 @@ import org.scalatest.Assertions + + import org.apache.spark.SparkUnsupportedOperationException + import org.apache.spark.io.CompressionCodec +-import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, Row} ++import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, IgnoreComet, Row} + import org.apache.spark.sql.catalyst.expressions.{BoundReference, GenericInternalRow} + import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning + import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil +@@ -525,11 +525,13 @@ abstract class StateDataSourceReadSuite extends StateDataSourceTestBase with Ass + } + } + +- test("flatMapGroupsWithState, state ver 1") { ++ test("flatMapGroupsWithState, state ver 1", ++ IgnoreComet("Ignored if Comet is enabled due to SPARK-49070.")) { + testFlatMapGroupsWithState(1) + } + +- test("flatMapGroupsWithState, state ver 2") { ++ test("flatMapGroupsWithState, state ver 2", ++ IgnoreComet("Ignored if Comet is enabled due to SPARK-49070.")) { + testFlatMapGroupsWithState(2) + } + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala index b8f3ea3c6f3..bbd44221288 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala @@ -2602,6 +3047,153 @@ index e05cb4d3c35..dc65a4fe18e 100644 }) } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateSuite.scala +index dea16e52989..55cdf47c4d5 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateSuite.scala +@@ -18,7 +18,7 @@ + package org.apache.spark.sql.streaming + + import org.apache.spark.SparkIllegalArgumentException +-import org.apache.spark.sql.Encoders ++import org.apache.spark.sql.{Encoders, IgnoreCometSuite} + import org.apache.spark.sql.execution.streaming.MemoryStream + import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithChangelogCheckpointingEnabled, RocksDBStateStoreProvider} + import org.apache.spark.sql.internal.SQLConf +@@ -128,7 +128,7 @@ class ToggleSaveAndEmitProcessor + } + + class TransformWithListStateSuite extends StreamTest +- with AlsoTestWithChangelogCheckpointingEnabled { ++ with AlsoTestWithChangelogCheckpointingEnabled with IgnoreCometSuite { + import testImplicits._ + + test("test appending null value in list state throw exception") { +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateTTLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateTTLSuite.scala +index 299a3346b2e..2213f3c52a6 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateTTLSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateTTLSuite.scala +@@ -19,7 +19,7 @@ package org.apache.spark.sql.streaming + + import java.time.Duration + +-import org.apache.spark.sql.Encoders ++import org.apache.spark.sql.{Encoders, IgnoreCometSuite} + import org.apache.spark.sql.execution.streaming.{ListStateImplWithTTL, MemoryStream} + import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider + import org.apache.spark.sql.internal.SQLConf +@@ -95,7 +95,7 @@ class ListStateTTLProcessor(ttlConfig: TTLConfig) + * Test suite for testing list state with TTL. + * We use the base TTL suite with a list state processor. + */ +-class TransformWithListStateTTLSuite extends TransformWithStateTTLTest { ++class TransformWithListStateTTLSuite extends TransformWithStateTTLTest with IgnoreCometSuite { + + import testImplicits._ + +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateTTLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateTTLSuite.scala +index bf46c802fde..623c3003430 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateTTLSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateTTLSuite.scala +@@ -19,7 +19,7 @@ package org.apache.spark.sql.streaming + + import java.time.Duration + +-import org.apache.spark.sql.Encoders ++import org.apache.spark.sql.{Encoders, IgnoreCometSuite} + import org.apache.spark.sql.execution.streaming.{MapStateImplWithTTL, MemoryStream} + import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider + import org.apache.spark.sql.internal.SQLConf +@@ -174,7 +174,7 @@ class MapStateTTLProcessor(ttlConfig: TTLConfig) + } + } + +-class TransformWithMapStateTTLSuite extends TransformWithStateTTLTest { ++class TransformWithMapStateTTLSuite extends TransformWithStateTTLTest with IgnoreCometSuite { + + import testImplicits._ + override def getProcessor(ttlConfig: TTLConfig): +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateChainingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateChainingSuite.scala +index 5388d6f1fb6..8aa11c5b875 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateChainingSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateChainingSuite.scala +@@ -21,7 +21,7 @@ import java.sql.Timestamp + import java.time.{Instant, LocalDateTime, ZoneId} + + import org.apache.spark.{SparkRuntimeException, SparkThrowable} +-import org.apache.spark.sql.AnalysisException ++import org.apache.spark.sql.{AnalysisException, IgnoreCometSuite} + import org.apache.spark.sql.catalyst.ExtendedAnalysisException + import org.apache.spark.sql.execution.streaming.{MemoryStream, StreamExecution} + import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider +@@ -106,7 +106,8 @@ case class AggEventRow( + window: Window, + count: Long) + +-class TransformWithStateChainingSuite extends StreamTest { ++class TransformWithStateChainingSuite extends StreamTest ++ with IgnoreCometSuite { + import testImplicits._ + + test("watermark is propagated correctly for next stateful operator" + +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala +index 0057af44d3e..51975748309 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala +@@ -22,7 +22,7 @@ import java.util.UUID + + import org.apache.spark.SparkRuntimeException + import org.apache.spark.internal.Logging +-import org.apache.spark.sql.{Dataset, Encoders} ++import org.apache.spark.sql.{Dataset, Encoders, IgnoreCometSuite} + import org.apache.spark.sql.catalyst.util.stringToFile + import org.apache.spark.sql.execution.streaming._ + import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithChangelogCheckpointingEnabled, RocksDBStateStoreProvider, StatefulProcessorCannotPerformOperationWithInvalidHandleState, StateStoreMultipleColumnFamiliesNotSupportedException} +@@ -307,9 +307,11 @@ class RunningCountStatefulProcessorWithError extends RunningCountStatefulProcess + + /** + * Class that adds tests for transformWithState stateful streaming operator ++ * ++ * Ignored if Comet is enabled due to SPARK-49070. + */ + class TransformWithStateSuite extends StateStoreMetricsTest +- with AlsoTestWithChangelogCheckpointingEnabled { ++ with AlsoTestWithChangelogCheckpointingEnabled with IgnoreCometSuite { + + import testImplicits._ + +@@ -786,7 +788,7 @@ class TransformWithStateSuite extends StateStoreMetricsTest + } + } + +-class TransformWithStateValidationSuite extends StateStoreMetricsTest { ++class TransformWithStateValidationSuite extends StateStoreMetricsTest with IgnoreCometSuite { + import testImplicits._ + + test("transformWithState - streaming with hdfsStateStoreProvider should fail") { +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithValueStateTTLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithValueStateTTLSuite.scala +index 54004b419f7..4e5b35aa0da 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithValueStateTTLSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithValueStateTTLSuite.scala +@@ -20,7 +20,7 @@ package org.apache.spark.sql.streaming + import java.time.Duration + + import org.apache.spark.internal.Logging +-import org.apache.spark.sql.Encoders ++import org.apache.spark.sql.{Encoders, IgnoreCometSuite} + import org.apache.spark.sql.execution.streaming.{MemoryStream, ValueStateImpl, ValueStateImplWithTTL} + import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider + import org.apache.spark.sql.internal.SQLConf +@@ -160,7 +160,8 @@ case class MultipleValueStatesTTLProcessor( + } + } + +-class TransformWithValueStateTTLSuite extends TransformWithStateTTLTest { ++class TransformWithValueStateTTLSuite extends TransformWithStateTTLTest ++ with IgnoreCometSuite{ + + import testImplicits._ + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala index af07aceaed1..ed0b5e6d9be 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala diff --git a/dev/ensure-jars-have-correct-contents.sh b/dev/ensure-jars-have-correct-contents.sh index 23d0be323..884e81774 100755 --- a/dev/ensure-jars-have-correct-contents.sh +++ b/dev/ensure-jars-have-correct-contents.sh @@ -94,6 +94,9 @@ allowed_expr+="|^org/apache/spark/CometPlugin.class$" allowed_expr+="|^org/apache/spark/CometDriverPlugin.*$" allowed_expr+="|^org/apache/spark/CometTaskMemoryManager.class$" allowed_expr+="|^org/apache/spark/CometTaskMemoryManager.*$" +# Custom ColumnarBatch +allowed_expr+="|^org/apache/spark/sql/vectorized/$" +allowed_expr+="|^org/apache/spark/sql/vectorized/CometColumnarBatch.class$" allowed_expr+=")" declare -i bad_artifacts=0 diff --git a/docs/source/contributor-guide/benchmarking.md b/docs/source/contributor-guide/benchmarking.md index b4d2f5f47..5f4f10912 100644 --- a/docs/source/contributor-guide/benchmarking.md +++ b/docs/source/contributor-guide/benchmarking.md @@ -67,7 +67,6 @@ $SPARK_HOME/bin/spark-submit \ --conf spark.comet.cast.allowIncompatible=true \ --conf spark.comet.exec.shuffle.enabled=true \ --conf spark.comet.exec.shuffle.mode=auto \ - --conf spark.comet.shuffle.enforceMode.enabled=true \ --conf spark.shuffle.manager=org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager \ tpcbench.py \ --benchmark tpch \ diff --git a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala index 29eb2f0ca..fe44091af 100644 --- a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala +++ b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala @@ -128,7 +128,6 @@ class CometExecIterator( nextBatch = getNextBatch() if (nextBatch.isEmpty) { - close() false } else { true diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala index f24624dcf..9ba850ca6 100644 --- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala +++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala @@ -1069,21 +1069,13 @@ object CometSparkSessionExtensions extends Logging { } private[comet] def isCometShuffleEnabled(conf: SQLConf): Boolean = - COMET_EXEC_SHUFFLE_ENABLED.get(conf) && isCometShuffleManagerEnabled(conf) && - // TODO: AQE coalesce partitions feature causes Comet shuffle memory leak. - // We should disable Comet shuffle when AQE coalesce partitions is enabled. - (!conf.coalesceShufflePartitionsEnabled || COMET_SHUFFLE_ENFORCE_MODE_ENABLED.get()) + COMET_EXEC_SHUFFLE_ENABLED.get(conf) && isCometShuffleManagerEnabled(conf) private[comet] def getCometShuffleNotEnabledReason(conf: SQLConf): Option[String] = { if (!COMET_EXEC_SHUFFLE_ENABLED.get(conf)) { Some(s"${COMET_EXEC_SHUFFLE_ENABLED.key} is not enabled") } else if (!isCometShuffleManagerEnabled(conf)) { Some(s"spark.shuffle.manager is not set to ${CometShuffleManager.getClass.getName}") - } else if (conf.coalesceShufflePartitionsEnabled && !COMET_SHUFFLE_ENFORCE_MODE_ENABLED - .get()) { - Some( - s"${SQLConf.COALESCE_PARTITIONS_ENABLED.key} is enabled and " + - s"${COMET_SHUFFLE_ENFORCE_MODE_ENABLED.key} is not enabled") } else { None } diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/explain.txt index d35b9049e..3752ee154 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/explain.txt @@ -1,47 +1,44 @@ == Physical Plan == -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) +* ColumnarToRow (40) ++- CometTakeOrderedAndProject (39) + +- CometProject (38) + +- CometBroadcastHashJoin (37) + :- CometProject (33) + : +- CometBroadcastHashJoin (32) + : :- CometProject (27) + : : +- CometBroadcastHashJoin (26) + : : :- CometFilter (12) + : : : +- CometHashAggregate (11) + : : : +- CometColumnarExchange (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) + : : +- CometBroadcastExchange (25) + : : +- CometFilter (24) + : : +- CometHashAggregate (23) + : : +- CometColumnarExchange (22) + : : +- CometHashAggregate (21) + : : +- CometHashAggregate (20) + : : +- CometColumnarExchange (19) + : : +- CometHashAggregate (18) + : : +- CometProject (17) + : : +- CometBroadcastHashJoin (16) + : : :- CometFilter (14) + : : : +- CometScan parquet spark_catalog.default.store_returns (13) + : : +- ReusedExchange (15) + : +- CometBroadcastExchange (31) + : +- CometProject (30) + : +- CometFilter (29) + : +- CometScan parquet spark_catalog.default.store (28) + +- CometBroadcastExchange (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.customer (34) (1) Scan parquet spark_catalog.default.store_returns @@ -89,199 +86,179 @@ 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))] -(10) ColumnarToRow [codegen id : 1] +(10) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(12) HashAggregate [codegen id : 7] +(11) CometHashAggregate 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] -(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) +(12) CometFilter +Input [3]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11] +Condition : isnotnull(ctr_total_return#11) -(14) 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] +(13) Scan parquet spark_catalog.default.store_returns +Output [4]: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14, sr_returned_date_sk#15] 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#15), dynamicpruningexpression(sr_returned_date_sk#15 IN dynamicpruning#16)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(15) 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) - -(16) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#18] - -(17) CometBroadcastHashJoin -Left output [4]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16] -Right output [1]: [d_date_sk#18] -Arguments: [sr_returned_date_sk#16], [d_date_sk#18], Inner, BuildRight - -(18) CometProject -Input [5]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16, d_date_sk#18] -Arguments: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15], [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15] - -(19) CometHashAggregate -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))] - -(20) ColumnarToRow [codegen id : 2] -Input [3]: [sr_customer_sk#13, sr_store_sk#14, sum#19] - -(21) Exchange -Input [3]: [sr_customer_sk#13, sr_store_sk#14, sum#19] -Arguments: hashpartitioning(sr_customer_sk#13, sr_store_sk#14, 5), ENSURE_REQUIREMENTS, [plan_id=2] - -(22) HashAggregate [codegen id : 3] -Input [3]: [sr_customer_sk#13, sr_store_sk#14, sum#19] -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#20, MakeDecimal(sum(UnscaledValue(sr_return_amt#15))#9,17,2) AS ctr_total_return#21] - -(23) HashAggregate [codegen id : 3] -Input [2]: [ctr_store_sk#20, ctr_total_return#21] -Keys [1]: [ctr_store_sk#20] -Functions [1]: [partial_avg(ctr_total_return#21)] -Aggregate Attributes [2]: [sum#22, count#23] -Results [3]: [ctr_store_sk#20, sum#24, count#25] - -(24) Exchange -Input [3]: [ctr_store_sk#20, sum#24, count#25] -Arguments: hashpartitioning(ctr_store_sk#20, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(25) HashAggregate [codegen id : 4] -Input [3]: [ctr_store_sk#20, sum#24, count#25] -Keys [1]: [ctr_store_sk#20] -Functions [1]: [avg(ctr_total_return#21)] -Aggregate Attributes [1]: [avg(ctr_total_return#21)#26] -Results [2]: [(avg(ctr_total_return#21)#26 * 1.2) AS (avg(ctr_total_return) * 1.2)#27, ctr_store_sk#20] - -(26) Filter [codegen id : 4] -Input [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_store_sk#20] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#27) - -(27) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_store_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=4] - -(28) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ctr_store_sk#11] -Right keys [1]: [ctr_store_sk#20] -Join type: Inner -Join condition: (cast(ctr_total_return#12 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#27) - -(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)#27, ctr_store_sk#20] - -(30) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#28, s_state#29] +(14) CometFilter +Input [4]: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14, sr_returned_date_sk#15] +Condition : isnotnull(sr_store_sk#13) + +(15) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#17] + +(16) CometBroadcastHashJoin +Left output [4]: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14, sr_returned_date_sk#15] +Right output [1]: [d_date_sk#17] +Arguments: [sr_returned_date_sk#15], [d_date_sk#17], Inner, BuildRight + +(17) CometProject +Input [5]: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14, sr_returned_date_sk#15, d_date_sk#17] +Arguments: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14], [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14] + +(18) CometHashAggregate +Input [3]: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14] +Keys [2]: [sr_customer_sk#12, sr_store_sk#13] +Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#14))] + +(19) CometColumnarExchange +Input [3]: [sr_customer_sk#12, sr_store_sk#13, sum#18] +Arguments: hashpartitioning(sr_customer_sk#12, sr_store_sk#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(20) CometHashAggregate +Input [3]: [sr_customer_sk#12, sr_store_sk#13, sum#18] +Keys [2]: [sr_customer_sk#12, sr_store_sk#13] +Functions [1]: [sum(UnscaledValue(sr_return_amt#14))] + +(21) CometHashAggregate +Input [2]: [ctr_store_sk#19, ctr_total_return#20] +Keys [1]: [ctr_store_sk#19] +Functions [1]: [partial_avg(ctr_total_return#20)] + +(22) CometColumnarExchange +Input [3]: [ctr_store_sk#19, sum#21, count#22] +Arguments: hashpartitioning(ctr_store_sk#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(23) CometHashAggregate +Input [3]: [ctr_store_sk#19, sum#21, count#22] +Keys [1]: [ctr_store_sk#19] +Functions [1]: [avg(ctr_total_return#20)] + +(24) CometFilter +Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_store_sk#19] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#23) + +(25) CometBroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_store_sk#19] +Arguments: [(avg(ctr_total_return) * 1.2)#23, ctr_store_sk#19] + +(26) CometBroadcastHashJoin +Left output [3]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11] +Right output [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_store_sk#19] +Arguments: [ctr_store_sk#10], [ctr_store_sk#19], Inner, (cast(ctr_total_return#11 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#23), BuildRight + +(27) CometProject +Input [5]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11, (avg(ctr_total_return) * 1.2)#23, ctr_store_sk#19] +Arguments: [ctr_customer_sk#9, ctr_store_sk#10], [ctr_customer_sk#9, ctr_store_sk#10] + +(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_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct -(31) CometFilter -Input [2]: [s_store_sk#28, s_state#29] -Condition : ((isnotnull(s_state#29) AND (s_state#29 = TN)) AND isnotnull(s_store_sk#28)) - -(32) CometProject -Input [2]: [s_store_sk#28, s_state#29] -Arguments: [s_store_sk#28], [s_store_sk#28] +(29) CometFilter +Input [2]: [s_store_sk#24, s_state#25] +Condition : ((isnotnull(s_state#25) AND (s_state#25 = TN)) AND isnotnull(s_store_sk#24)) -(33) ColumnarToRow [codegen id : 5] -Input [1]: [s_store_sk#28] +(30) CometProject +Input [2]: [s_store_sk#24, s_state#25] +Arguments: [s_store_sk#24], [s_store_sk#24] -(34) BroadcastExchange -Input [1]: [s_store_sk#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(31) CometBroadcastExchange +Input [1]: [s_store_sk#24] +Arguments: [s_store_sk#24] -(35) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ctr_store_sk#11] -Right keys [1]: [s_store_sk#28] -Join type: Inner -Join condition: None +(32) CometBroadcastHashJoin +Left output [2]: [ctr_customer_sk#9, ctr_store_sk#10] +Right output [1]: [s_store_sk#24] +Arguments: [ctr_store_sk#10], [s_store_sk#24], Inner, BuildRight -(36) Project [codegen id : 7] -Output [1]: [ctr_customer_sk#10] -Input [3]: [ctr_customer_sk#10, ctr_store_sk#11, s_store_sk#28] +(33) CometProject +Input [3]: [ctr_customer_sk#9, ctr_store_sk#10, s_store_sk#24] +Arguments: [ctr_customer_sk#9], [ctr_customer_sk#9] -(37) Scan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#30, c_customer_id#31] +(34) Scan parquet spark_catalog.default.customer +Output [2]: [c_customer_sk#26, c_customer_id#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(38) CometFilter -Input [2]: [c_customer_sk#30, c_customer_id#31] -Condition : isnotnull(c_customer_sk#30) +(35) CometFilter +Input [2]: [c_customer_sk#26, c_customer_id#27] +Condition : isnotnull(c_customer_sk#26) -(39) ColumnarToRow [codegen id : 6] -Input [2]: [c_customer_sk#30, c_customer_id#31] +(36) CometBroadcastExchange +Input [2]: [c_customer_sk#26, c_customer_id#27] +Arguments: [c_customer_sk#26, c_customer_id#27] -(40) BroadcastExchange -Input [2]: [c_customer_sk#30, c_customer_id#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +(37) CometBroadcastHashJoin +Left output [1]: [ctr_customer_sk#9] +Right output [2]: [c_customer_sk#26, c_customer_id#27] +Arguments: [ctr_customer_sk#9], [c_customer_sk#26], Inner, BuildRight -(41) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ctr_customer_sk#10] -Right keys [1]: [c_customer_sk#30] -Join type: Inner -Join condition: None +(38) CometProject +Input [3]: [ctr_customer_sk#9, c_customer_sk#26, c_customer_id#27] +Arguments: [c_customer_id#27], [c_customer_id#27] -(42) Project [codegen id : 7] -Output [1]: [c_customer_id#31] -Input [3]: [ctr_customer_sk#10, c_customer_sk#30, c_customer_id#31] +(39) CometTakeOrderedAndProject +Input [1]: [c_customer_id#27] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#27 ASC NULLS FIRST], output=[c_customer_id#27]), [c_customer_id#27], 100, [c_customer_id#27 ASC NULLS FIRST], [c_customer_id#27] -(43) TakeOrderedAndProject -Input [1]: [c_customer_id#31] -Arguments: 100, [c_customer_id#31 ASC NULLS FIRST], [c_customer_id#31] +(40) ColumnarToRow [codegen id : 1] +Input [1]: [c_customer_id#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (48) -+- * ColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan parquet spark_catalog.default.date_dim (44) +BroadcastExchange (45) ++- * ColumnarToRow (44) + +- CometProject (43) + +- CometFilter (42) + +- CometScan parquet spark_catalog.default.date_dim (41) -(44) Scan parquet spark_catalog.default.date_dim +(41) 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 -(45) CometFilter +(42) 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)) -(46) CometProject +(43) CometProject Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(47) ColumnarToRow [codegen id : 1] +(44) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(48) BroadcastExchange +(45) BroadcastExchange Input [1]: [d_date_sk#6] -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=4] -Subquery:2 Hosting operator id = 14 Hosting Expression = sr_returned_date_sk#16 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 13 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/simplified.txt index d6f75837b..28b22a2e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/simplified.txt @@ -1,69 +1,51 @@ -TakeOrderedAndProject [c_customer_id] - WholeStageCodegen (7) - Project [c_customer_id] - BroadcastHashJoin [ctr_customer_sk,c_customer_sk] - Project [ctr_customer_sk] - BroadcastHashJoin [ctr_store_sk,s_store_sk] - Project [ctr_customer_sk,ctr_store_sk] - BroadcastHashJoin [ctr_store_sk,ctr_store_sk,ctr_total_return,(avg(ctr_total_return) * 1.2)] - Filter [ctr_total_return] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [sr_customer_sk,sr_store_sk,sum,sr_return_amt] - CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] - CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - 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] #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] #6 - WholeStageCodegen (2) +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [c_customer_id] + CometProject [c_customer_id] + CometBroadcastHashJoin [ctr_customer_sk,c_customer_sk,c_customer_id] + CometProject [ctr_customer_sk] + CometBroadcastHashJoin [ctr_customer_sk,ctr_store_sk,s_store_sk] + CometProject [ctr_customer_sk,ctr_store_sk] + CometBroadcastHashJoin [ctr_customer_sk,ctr_store_sk,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_store_sk] + CometFilter [ctr_customer_sk,ctr_store_sk,ctr_total_return] + CometHashAggregate [ctr_customer_sk,ctr_store_sk,ctr_total_return,sr_customer_sk,sr_store_sk,sum,sum(UnscaledValue(sr_return_amt))] + CometColumnarExchange [sr_customer_sk,sr_store_sk] #1 + CometHashAggregate [sr_customer_sk,sr_store_sk,sum,sr_return_amt] + CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] + CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] + CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [sr_customer_sk,sr_store_sk,sum,sr_return_amt] - CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] - CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_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 - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id] + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_store_sk] #4 + CometFilter [(avg(ctr_total_return) * 1.2),ctr_store_sk] + CometHashAggregate [(avg(ctr_total_return) * 1.2),ctr_store_sk,sum,count,avg(ctr_total_return)] + CometColumnarExchange [ctr_store_sk] #5 + CometHashAggregate [ctr_store_sk,sum,count,ctr_total_return] + CometHashAggregate [ctr_store_sk,ctr_total_return,sr_customer_sk,sr_store_sk,sum,sum(UnscaledValue(sr_return_amt))] + CometColumnarExchange [sr_customer_sk,sr_store_sk] #6 + CometHashAggregate [sr_customer_sk,sr_store_sk,sum,sr_return_amt] + CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] + CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] + CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_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 + ReusedExchange [d_date_sk] #3 + CometBroadcastExchange [s_store_sk] #7 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometBroadcastExchange [c_customer_sk,c_customer_id] #8 + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/explain.txt index 4a29b7260..6f8c539fb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/explain.txt @@ -1,49 +1,51 @@ == Physical Plan == -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) +TakeOrderedAndProject (47) ++- * HashAggregate (46) + +- * ColumnarToRow (45) + +- CometColumnarExchange (44) + +- RowToColumnar (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 @@ -243,50 +245,56 @@ Functions [1]: [partial_count(1)] 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] -(43) Exchange +(43) RowToColumnar 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] -(44) HashAggregate [codegen id : 6] +(44) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=5] + +(45) ColumnarToRow [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] + +(46) 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)#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] -(45) TakeOrderedAndProject +(47) 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 = 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) +BroadcastExchange (52) ++- * ColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) -(46) Scan parquet spark_catalog.default.date_dim +(48) 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 -(47) CometFilter +(49) 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)) -(48) CometProject +(50) CometProject Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(49) ColumnarToRow [codegen id : 1] +(51) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(50) BroadcastExchange +(52) BroadcastExchange Input [1]: [d_date_sk#9] 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-spark3_5/q10/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/simplified.txt index efd4b187d..a3ab6bbbb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/simplified.txt @@ -1,71 +1,73 @@ 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 (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 (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] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] - CometFilter [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] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,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 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometColumnarExchange [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 + RowToColumnar + 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] + Project [c_current_cdemo_sk] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] + Filter [exists,exists] + BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,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 + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [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] + CometBroadcastExchange [ss_customer_sk] #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,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 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [cs_ship_customer_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,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 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_county] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) + BroadcastExchange #8 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [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] - 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] + CometFilter [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] + 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-v1_4-spark3_5/q11/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/explain.txt index 1c5d5222e..ac0d82f80 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/explain.txt @@ -1,76 +1,73 @@ == Physical Plan == -TakeOrderedAndProject (72) -+- * Project (71) - +- * BroadcastHashJoin Inner BuildRight (70) - :- * 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) - +- * 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) +* ColumnarToRow (69) ++- CometTakeOrderedAndProject (68) + +- CometProject (67) + +- CometBroadcastHashJoin (66) + :- CometProject (51) + : +- CometBroadcastHashJoin (50) + : :- CometProject (34) + : : +- CometBroadcastHashJoin (33) + : : :- CometFilter (16) + : : : +- CometHashAggregate (15) + : : : +- CometColumnarExchange (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) + : : +- CometBroadcastExchange (32) + : : +- CometHashAggregate (31) + : : +- CometColumnarExchange (30) + : : +- CometHashAggregate (29) + : : +- CometProject (28) + : : +- CometBroadcastHashJoin (27) + : : :- CometProject (23) + : : : +- CometBroadcastHashJoin (22) + : : : :- CometFilter (18) + : : : : +- CometScan parquet spark_catalog.default.customer (17) + : : : +- CometBroadcastExchange (21) + : : : +- CometFilter (20) + : : : +- CometScan parquet spark_catalog.default.store_sales (19) + : : +- CometBroadcastExchange (26) + : : +- CometFilter (25) + : : +- CometScan parquet spark_catalog.default.date_dim (24) + : +- CometBroadcastExchange (49) + : +- CometFilter (48) + : +- CometHashAggregate (47) + : +- CometColumnarExchange (46) + : +- CometHashAggregate (45) + : +- CometProject (44) + : +- CometBroadcastHashJoin (43) + : :- CometProject (41) + : : +- CometBroadcastHashJoin (40) + : : :- CometFilter (36) + : : : +- CometScan parquet spark_catalog.default.customer (35) + : : +- CometBroadcastExchange (39) + : : +- CometFilter (38) + : : +- CometScan parquet spark_catalog.default.web_sales (37) + : +- ReusedExchange (42) + +- CometBroadcastExchange (65) + +- CometHashAggregate (64) + +- CometColumnarExchange (63) + +- CometHashAggregate (62) + +- CometProject (61) + +- CometBroadcastHashJoin (60) + :- CometProject (58) + : +- CometBroadcastHashJoin (57) + : :- CometFilter (53) + : : +- CometScan parquet spark_catalog.default.customer (52) + : +- CometBroadcastExchange (56) + : +- CometFilter (55) + : +- CometScan parquet spark_catalog.default.web_sales (54) + +- ReusedExchange (59) (1) Scan parquet spark_catalog.default.customer @@ -138,341 +135,321 @@ Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_fl 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)))] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(16) HashAggregate [codegen id : 8] +(15) CometHashAggregate 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)))#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] -(17) Filter [codegen id : 8] -Input [2]: [customer_id#18, year_total#19] -Condition : (isnotnull(year_total#19) AND (year_total#19 > 0.00)) +(16) CometFilter +Input [2]: [customer_id#17, year_total#18] +Condition : (isnotnull(year_total#18) AND (year_total#18 > 0.00)) -(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] +(17) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(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)) +(18) CometFilter +Input [8]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26] +Condition : (isnotnull(c_customer_sk#19) AND isnotnull(c_customer_id#20)) -(20) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] +(19) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_customer_sk#27, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#31), dynamicpruningexpression(ss_sold_date_sk#31 IN dynamicpruning#32)] +PartitionFilters: [isnotnull(ss_sold_date_sk#30), dynamicpruningexpression(ss_sold_date_sk#30 IN dynamicpruning#31)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(21) CometFilter -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) +(20) CometFilter +Input [4]: [ss_customer_sk#27, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30] +Condition : isnotnull(ss_customer_sk#27) -(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] +(21) CometBroadcastExchange +Input [4]: [ss_customer_sk#27, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30] +Arguments: [ss_customer_sk#27, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30] -(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 +(22) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26] +Right output [4]: [ss_customer_sk#27, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30] +Arguments: [c_customer_sk#19], [ss_customer_sk#27], Inner, BuildRight -(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] +(23) CometProject +Input [12]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, ss_customer_sk#27, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30] +Arguments: [c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30], [c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30] -(25) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_year#34] +(24) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#32, d_year#33] 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 -(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 - -(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] - -(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] +(25) CometFilter +Input [2]: [d_date_sk#32, d_year#33] +Condition : ((isnotnull(d_year#33) AND (d_year#33 = 2002)) AND isnotnull(d_date_sk#32)) + +(26) CometBroadcastExchange +Input [2]: [d_date_sk#32, d_year#33] +Arguments: [d_date_sk#32, d_year#33] + +(27) CometBroadcastHashJoin +Left output [10]: [c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30] +Right output [2]: [d_date_sk#32, d_year#33] +Arguments: [ss_sold_date_sk#30], [d_date_sk#32], Inner, BuildRight + +(28) CometProject +Input [12]: [c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30, d_date_sk#32, d_year#33] +Arguments: [c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, ss_ext_discount_amt#28, ss_ext_list_price#29, d_year#33], [c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, ss_ext_discount_amt#28, ss_ext_list_price#29, d_year#33] + +(29) CometHashAggregate +Input [10]: [c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, ss_ext_discount_amt#28, ss_ext_list_price#29, d_year#33] +Keys [8]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#33, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26] +Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#29 - ss_ext_discount_amt#28)))] + +(30) CometColumnarExchange +Input [9]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#33, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, sum#34] +Arguments: hashpartitioning(c_customer_id#20, c_first_name#21, c_last_name#22, d_year#33, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(31) CometHashAggregate +Input [9]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#33, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, sum#34] +Keys [8]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#33, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26] +Functions [1]: [sum(UnscaledValue((ss_ext_list_price#29 - ss_ext_discount_amt#28)))] + +(32) CometBroadcastExchange +Input [3]: [customer_id#35, customer_preferred_cust_flag#36, year_total#37] +Arguments: [customer_id#35, customer_preferred_cust_flag#36, year_total#37] + +(33) CometBroadcastHashJoin +Left output [2]: [customer_id#17, year_total#18] +Right output [3]: [customer_id#35, customer_preferred_cust_flag#36, year_total#37] +Arguments: [customer_id#17], [customer_id#35], Inner, BuildRight + +(34) CometProject +Input [5]: [customer_id#17, year_total#18, customer_id#35, customer_preferred_cust_flag#36, year_total#37] +Arguments: [customer_id#17, year_total#18, customer_preferred_cust_flag#36, year_total#37], [customer_id#17, year_total#18, customer_preferred_cust_flag#36, year_total#37] + +(35) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#38, c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#43, c_login#44, c_email_address#45] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(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)) +(36) CometFilter +Input [8]: [c_customer_sk#38, c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#43, c_login#44, c_email_address#45] +Condition : (isnotnull(c_customer_sk#38) AND isnotnull(c_customer_id#39)) -(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] +(37) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_bill_customer_sk#46, ws_ext_discount_amt#47, ws_ext_list_price#48, ws_sold_date_sk#49] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#50), dynamicpruningexpression(ws_sold_date_sk#50 IN dynamicpruning#51)] +PartitionFilters: [isnotnull(ws_sold_date_sk#49), dynamicpruningexpression(ws_sold_date_sk#49 IN dynamicpruning#50)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(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) +(38) CometFilter +Input [4]: [ws_bill_customer_sk#46, ws_ext_discount_amt#47, ws_ext_list_price#48, ws_sold_date_sk#49] +Condition : isnotnull(ws_bill_customer_sk#46) + +(39) CometBroadcastExchange +Input [4]: [ws_bill_customer_sk#46, ws_ext_discount_amt#47, ws_ext_list_price#48, ws_sold_date_sk#49] +Arguments: [ws_bill_customer_sk#46, ws_ext_discount_amt#47, ws_ext_list_price#48, ws_sold_date_sk#49] + +(40) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#38, c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#43, c_login#44, c_email_address#45] +Right output [4]: [ws_bill_customer_sk#46, ws_ext_discount_amt#47, ws_ext_list_price#48, ws_sold_date_sk#49] +Arguments: [c_customer_sk#38], [ws_bill_customer_sk#46], Inner, BuildRight -(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) CometProject +Input [12]: [c_customer_sk#38, c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#43, c_login#44, c_email_address#45, ws_bill_customer_sk#46, ws_ext_discount_amt#47, ws_ext_list_price#48, ws_sold_date_sk#49] +Arguments: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#43, c_login#44, c_email_address#45, ws_ext_discount_amt#47, ws_ext_list_price#48, ws_sold_date_sk#49], [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#43, c_login#44, c_email_address#45, ws_ext_discount_amt#47, ws_ext_list_price#48, ws_sold_date_sk#49] -(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) ReusedExchange [Reuses operator id: 10] +Output [2]: [d_date_sk#51, d_year#52] -(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) CometBroadcastHashJoin +Left output [10]: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#43, c_login#44, c_email_address#45, ws_ext_discount_amt#47, ws_ext_list_price#48, ws_sold_date_sk#49] +Right output [2]: [d_date_sk#51, d_year#52] +Arguments: [ws_sold_date_sk#49], [d_date_sk#51], Inner, BuildRight -(44) ReusedExchange [Reuses operator id: 10] -Output [2]: [d_date_sk#52, d_year#53] +(44) CometProject +Input [12]: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#43, c_login#44, c_email_address#45, ws_ext_discount_amt#47, ws_ext_list_price#48, ws_sold_date_sk#49, d_date_sk#51, d_year#52] +Arguments: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#43, c_login#44, c_email_address#45, ws_ext_discount_amt#47, ws_ext_list_price#48, d_year#52], [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#43, c_login#44, c_email_address#45, ws_ext_discount_amt#47, ws_ext_list_price#48, d_year#52] -(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) CometHashAggregate +Input [10]: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#43, c_login#44, c_email_address#45, ws_ext_discount_amt#47, ws_ext_list_price#48, d_year#52] +Keys [8]: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#43, c_login#44, c_email_address#45, d_year#52] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#48 - ws_ext_discount_amt#47)))] -(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] +(46) CometColumnarExchange +Input [9]: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#43, c_login#44, c_email_address#45, d_year#52, sum#53] +Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#43, c_login#44, c_email_address#45, d_year#52, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (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] - -(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 - -(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] - -(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] +Input [9]: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#43, c_login#44, c_email_address#45, d_year#52, sum#53] +Keys [8]: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#43, c_login#44, c_email_address#45, d_year#52] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#48 - ws_ext_discount_amt#47)))] + +(48) CometFilter +Input [2]: [customer_id#54, year_total#55] +Condition : (isnotnull(year_total#55) AND (year_total#55 > 0.00)) + +(49) CometBroadcastExchange +Input [2]: [customer_id#54, year_total#55] +Arguments: [customer_id#54, year_total#55] + +(50) CometBroadcastHashJoin +Left output [4]: [customer_id#17, year_total#18, customer_preferred_cust_flag#36, year_total#37] +Right output [2]: [customer_id#54, year_total#55] +Arguments: [customer_id#17], [customer_id#54], Inner, BuildRight + +(51) CometProject +Input [6]: [customer_id#17, year_total#18, customer_preferred_cust_flag#36, year_total#37, customer_id#54, year_total#55] +Arguments: [customer_id#17, year_total#18, customer_preferred_cust_flag#36, year_total#37, year_total#55], [customer_id#17, year_total#18, customer_preferred_cust_flag#36, year_total#37, year_total#55] + +(52) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#56, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(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)) +(53) CometFilter +Input [8]: [c_customer_sk#56, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63] +Condition : (isnotnull(c_customer_sk#56) AND isnotnull(c_customer_id#57)) -(57) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] +(54) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#69), dynamicpruningexpression(ws_sold_date_sk#69 IN dynamicpruning#70)] +PartitionFilters: [isnotnull(ws_sold_date_sk#67), dynamicpruningexpression(ws_sold_date_sk#67 IN dynamicpruning#68)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(58) CometFilter -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) +(55) CometFilter +Input [4]: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] +Condition : isnotnull(ws_bill_customer_sk#64) + +(56) CometBroadcastExchange +Input [4]: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] +Arguments: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] + +(57) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#56, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63] +Right output [4]: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] +Arguments: [c_customer_sk#56], [ws_bill_customer_sk#64], Inner, BuildRight + +(58) CometProject +Input [12]: [c_customer_sk#56, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63, ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] +Arguments: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67], [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] -(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] +(59) ReusedExchange [Reuses operator id: 26] +Output [2]: [d_date_sk#69, d_year#70] (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 +Left output [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] +Right output [2]: [d_date_sk#69, d_year#70] +Arguments: [ws_sold_date_sk#67], [d_date_sk#69], Inner, BuildRight (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) ReusedExchange [Reuses operator id: 27] -Output [2]: [d_date_sk#71, d_year#72] - -(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) 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) 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) 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#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 : 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#74, year_total#75] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] - -(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#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 : 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#37] -Arguments: 100, [customer_preferred_cust_flag#37 ASC NULLS FIRST], [customer_preferred_cust_flag#37] +Input [12]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67, d_date_sk#69, d_year#70] +Arguments: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, d_year#70], [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, d_year#70] + +(62) CometHashAggregate +Input [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, d_year#70] +Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63, d_year#70] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#66 - ws_ext_discount_amt#65)))] + +(63) CometColumnarExchange +Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63, d_year#70, sum#71] +Arguments: hashpartitioning(c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63, d_year#70, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(64) CometHashAggregate +Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63, d_year#70, sum#71] +Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63, d_year#70] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#66 - ws_ext_discount_amt#65)))] + +(65) CometBroadcastExchange +Input [2]: [customer_id#72, year_total#73] +Arguments: [customer_id#72, year_total#73] + +(66) CometBroadcastHashJoin +Left output [5]: [customer_id#17, year_total#18, customer_preferred_cust_flag#36, year_total#37, year_total#55] +Right output [2]: [customer_id#72, year_total#73] +Arguments: [customer_id#17], [customer_id#72], Inner, (CASE WHEN (year_total#55 > 0.00) THEN (year_total#73 / year_total#55) END > CASE WHEN (year_total#18 > 0.00) THEN (year_total#37 / year_total#18) END), BuildRight + +(67) CometProject +Input [7]: [customer_id#17, year_total#18, customer_preferred_cust_flag#36, year_total#37, year_total#55, customer_id#72, year_total#73] +Arguments: [customer_preferred_cust_flag#36], [customer_preferred_cust_flag#36] + +(68) CometTakeOrderedAndProject +Input [1]: [customer_preferred_cust_flag#36] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_preferred_cust_flag#36 ASC NULLS FIRST], output=[customer_preferred_cust_flag#36]), [customer_preferred_cust_flag#36], 100, [customer_preferred_cust_flag#36 ASC NULLS FIRST], [customer_preferred_cust_flag#36] + +(69) ColumnarToRow [codegen id : 1] +Input [1]: [customer_preferred_cust_flag#36] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (76) -+- * ColumnarToRow (75) - +- CometFilter (74) - +- CometScan parquet spark_catalog.default.date_dim (73) +BroadcastExchange (73) ++- * ColumnarToRow (72) + +- CometFilter (71) + +- CometScan parquet spark_catalog.default.date_dim (70) -(73) Scan parquet spark_catalog.default.date_dim +(70) 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 -(74) CometFilter +(71) 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)) -(75) ColumnarToRow [codegen id : 1] +(72) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#14, d_year#15] -(76) BroadcastExchange +(73) BroadcastExchange Input [2]: [d_date_sk#14, d_year#15] -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=5] -Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#31 IN dynamicpruning#32 -BroadcastExchange (80) -+- * ColumnarToRow (79) - +- CometFilter (78) - +- CometScan parquet spark_catalog.default.date_dim (77) +Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#31 +BroadcastExchange (77) ++- * ColumnarToRow (76) + +- CometFilter (75) + +- CometScan parquet spark_catalog.default.date_dim (74) -(77) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_year#34] +(74) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#32, d_year#33] 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#33, d_year#34] -Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2002)) AND isnotnull(d_date_sk#33)) +(75) CometFilter +Input [2]: [d_date_sk#32, d_year#33] +Condition : ((isnotnull(d_year#33) AND (d_year#33 = 2002)) AND isnotnull(d_date_sk#32)) -(79) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#33, d_year#34] +(76) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#32, d_year#33] -(80) BroadcastExchange -Input [2]: [d_date_sk#33, d_year#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] +(77) BroadcastExchange +Input [2]: [d_date_sk#32, d_year#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#50 IN dynamicpruning#13 +Subquery:3 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#49 IN dynamicpruning#13 -Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#69 IN dynamicpruning#32 +Subquery:4 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#67 IN dynamicpruning#31 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/simplified.txt index 127a5a291..83ffbd0a6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/simplified.txt @@ -1,107 +1,87 @@ -TakeOrderedAndProject [customer_preferred_cust_flag] - 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] - BroadcastHashJoin [customer_id,customer_id] - Project [customer_id,year_total,customer_preferred_cust_flag,year_total] - BroadcastHashJoin [customer_id,customer_id] - Filter [year_total] - 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 (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,sum,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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_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_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - 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,sum,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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 - CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_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 #8 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - 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] #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,sum,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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 - CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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 #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] #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,sum,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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [customer_preferred_cust_flag] + CometProject [customer_preferred_cust_flag] + CometBroadcastHashJoin [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total,customer_id,year_total] + CometProject [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total] + CometBroadcastHashJoin [customer_id,year_total,customer_preferred_cust_flag,year_total,customer_id,year_total] + CometProject [customer_id,year_total,customer_preferred_cust_flag,year_total] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_preferred_cust_flag,year_total] + CometFilter [customer_id,year_total] + CometHashAggregate [customer_id,year_total,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)))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 - CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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 + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_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_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [customer_id,customer_preferred_cust_flag,year_total] #5 + CometHashAggregate [customer_id,customer_preferred_cust_flag,year_total,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)))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #6 + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + 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 [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_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 #8 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [customer_id,year_total] #10 + CometFilter [customer_id,year_total] + CometHashAggregate [customer_id,year_total,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)))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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 + CometBroadcastExchange [customer_id,year_total] #13 + CometHashAggregate [customer_id,year_total,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)))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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-spark3_5/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/explain.txt index 160f0c593..40c6660a9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/explain.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject (22) +- * Project (21) +- Window (20) - +- * Sort (19) - +- Exchange (18) - +- * HashAggregate (17) - +- Exchange (16) - +- * ColumnarToRow (15) + +- * ColumnarToRow (19) + +- CometSort (18) + +- CometColumnarExchange (17) + +- CometHashAggregate (16) + +- CometColumnarExchange (15) +- CometHashAggregate (14) +- CometProject (13) +- CometBroadcastHashJoin (12) @@ -92,39 +92,37 @@ Input [6]: [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, 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))] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(16) CometHashAggregate 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] -(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=2] +(17) CometColumnarExchange +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(18) CometSort +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] +Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6], [i_class#9 ASC NULLS FIRST] -(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 +(19) ColumnarToRow [codegen id : 1] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] (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] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] +Arguments: [sum(_w0#15) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#16], [i_class#9] -(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] +(21) Project [codegen id : 2] +Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, ((_w0#15 * 100) / _we0#16) AS revenueratio#17, i_item_id#6] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6, _we0#16] (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] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17, 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#17 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/simplified.txt index 4c2d3f3d7..3e0fd16a4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/simplified.txt @@ -1,38 +1,34 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (4) + WholeStageCodegen (2) Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] InputAdapter Window [_w0,i_class] - WholeStageCodegen (3) - Sort [i_class] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [i_class] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ws_ext_sales_price] - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometColumnarExchange [i_class] #1 + CometHashAggregate [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ws_ext_sales_price] + CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + 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-spark3_5/q13/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/explain.txt index 1cda12261..b05df2927 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* HashAggregate (33) -+- Exchange (32) - +- * ColumnarToRow (31) +* ColumnarToRow (33) ++- CometHashAggregate (32) + +- CometColumnarExchange (31) +- CometHashAggregate (30) +- CometProject (29) +- CometBroadcastHashJoin (28) @@ -179,19 +179,17 @@ 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))] -(31) ColumnarToRow [codegen id : 1] +(31) CometColumnarExchange Input [7]: [sum#23, count#24, sum#25, count#26, sum#27, count#28, sum#29] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(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] +(32) CometHashAggregate 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)#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] + +(33) ColumnarToRow [codegen id : 1] +Input [4]: [avg(ss_quantity)#30, avg(ss_ext_sales_price)#31, avg(ss_ext_wholesale_cost)#32, sum(ss_ext_wholesale_cost)#33] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/simplified.txt index 98d995db0..0f438f153 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/simplified.txt @@ -1,45 +1,43 @@ -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] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [sum,count,sum,count,sum,count,sum,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,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status,hd_demo_sk,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,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_demo_sk,cd_marital_status,cd_education_status] - CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost] - CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,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_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,ca_address_sk,ca_state] - 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_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,s_store_sk] - CometFilter [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] - 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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk] #3 - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange [ca_address_sk,ca_state] #4 - CometProject [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #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 [hd_demo_sk,hd_dep_count] #7 - CometFilter [hd_demo_sk,hd_dep_count] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] + CometHashAggregate [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,avg(ss_quantity),avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),sum(UnscaledValue(ss_ext_wholesale_cost))] + CometColumnarExchange #1 + CometHashAggregate [sum,count,sum,count,sum,count,sum,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,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status,hd_demo_sk,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,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_demo_sk,cd_marital_status,cd_education_status] + CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,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_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,ca_address_sk,ca_state] + 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_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,s_store_sk] + CometFilter [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] + 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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [s_store_sk] #3 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] + CometBroadcastExchange [ca_address_sk,ca_state] #4 + CometProject [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #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 [hd_demo_sk,hd_dep_count] #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-spark3_5/q14a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/explain.txt index 01b77a0d3..81c7b9c42 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/explain.txt @@ -1,109 +1,106 @@ == Physical Plan == -TakeOrderedAndProject (105) -+- * HashAggregate (104) - +- Exchange (103) - +- * HashAggregate (102) - +- * Expand (101) - +- Union (100) - :- * Project (67) - : +- * Filter (66) - : +- * HashAggregate (65) - : +- Exchange (64) - : +- * HashAggregate (63) - : +- * Project (62) - : +- * BroadcastHashJoin Inner BuildRight (61) - : :- * Project (59) - : : +- * BroadcastHashJoin Inner BuildRight (58) - : : :- * BroadcastHashJoin LeftSemi BuildRight (51) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (50) - : : : +- * Project (49) - : : : +- * BroadcastHashJoin Inner BuildRight (48) - : : : :- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.item (4) - : : : +- BroadcastExchange (47) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) - : : : :- * HashAggregate (35) - : : : : +- Exchange (34) - : : : : +- * 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) - : : : +- * 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) - : : : +- CometFilter (53) - : : : +- CometScan parquet spark_catalog.default.item (52) - : : +- ReusedExchange (55) - : +- ReusedExchange (60) - :- * Project (83) - : +- * Filter (82) - : +- * HashAggregate (81) - : +- Exchange (80) - : +- * HashAggregate (79) - : +- * Project (78) - : +- * BroadcastHashJoin Inner BuildRight (77) - : :- * Project (75) - : : +- * BroadcastHashJoin Inner BuildRight (74) - : : :- * BroadcastHashJoin LeftSemi BuildRight (72) - : : : :- * ColumnarToRow (70) - : : : : +- CometFilter (69) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (68) - : : : +- ReusedExchange (71) - : : +- ReusedExchange (73) - : +- ReusedExchange (76) - +- * Project (99) - +- * Filter (98) - +- * HashAggregate (97) - +- Exchange (96) - +- * HashAggregate (95) - +- * Project (94) - +- * BroadcastHashJoin Inner BuildRight (93) - :- * Project (91) - : +- * BroadcastHashJoin Inner BuildRight (90) - : :- * BroadcastHashJoin LeftSemi BuildRight (88) - : : :- * ColumnarToRow (86) - : : : +- CometFilter (85) - : : : +- CometScan parquet spark_catalog.default.web_sales (84) - : : +- ReusedExchange (87) - : +- ReusedExchange (89) - +- ReusedExchange (92) +* ColumnarToRow (102) ++- CometTakeOrderedAndProject (101) + +- CometHashAggregate (100) + +- CometColumnarExchange (99) + +- CometHashAggregate (98) + +- CometExpand (97) + +- CometUnion (96) + :- CometProject (65) + : +- CometFilter (64) + : +- CometHashAggregate (63) + : +- CometColumnarExchange (62) + : +- CometHashAggregate (61) + : +- CometProject (60) + : +- CometBroadcastHashJoin (59) + : :- CometProject (54) + : : +- CometBroadcastHashJoin (53) + : : :- CometBroadcastHashJoin (47) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (46) + : : : +- CometProject (45) + : : : +- CometBroadcastHashJoin (44) + : : : :- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.item (3) + : : : +- CometBroadcastExchange (43) + : : : +- CometBroadcastHashJoin (42) + : : : :- CometHashAggregate (32) + : : : : +- CometColumnarExchange (31) + : : : : +- CometHashAggregate (30) + : : : : +- CometProject (29) + : : : : +- CometBroadcastHashJoin (28) + : : : : :- CometProject (26) + : : : : : +- CometBroadcastHashJoin (25) + : : : : : :- CometFilter (6) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (5) + : : : : : +- CometBroadcastExchange (24) + : : : : : +- CometBroadcastHashJoin (23) + : : : : : :- CometFilter (8) + : : : : : : +- CometScan parquet spark_catalog.default.item (7) + : : : : : +- CometBroadcastExchange (22) + : : : : : +- CometProject (21) + : : : : : +- CometBroadcastHashJoin (20) + : : : : : :- CometProject (15) + : : : : : : +- CometBroadcastHashJoin (14) + : : : : : : :- CometFilter (10) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (9) + : : : : : : +- CometBroadcastExchange (13) + : : : : : : +- CometFilter (12) + : : : : : : +- CometScan parquet spark_catalog.default.item (11) + : : : : : +- CometBroadcastExchange (19) + : : : : : +- CometProject (18) + : : : : : +- CometFilter (17) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (16) + : : : : +- ReusedExchange (27) + : : : +- CometBroadcastExchange (41) + : : : +- CometProject (40) + : : : +- CometBroadcastHashJoin (39) + : : : :- CometProject (37) + : : : : +- CometBroadcastHashJoin (36) + : : : : :- CometFilter (34) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (33) + : : : : +- ReusedExchange (35) + : : : +- ReusedExchange (38) + : : +- CometBroadcastExchange (52) + : : +- CometBroadcastHashJoin (51) + : : :- CometFilter (49) + : : : +- CometScan parquet spark_catalog.default.item (48) + : : +- ReusedExchange (50) + : +- CometBroadcastExchange (58) + : +- CometProject (57) + : +- CometFilter (56) + : +- CometScan parquet spark_catalog.default.date_dim (55) + :- CometProject (80) + : +- CometFilter (79) + : +- CometHashAggregate (78) + : +- CometColumnarExchange (77) + : +- CometHashAggregate (76) + : +- CometProject (75) + : +- CometBroadcastHashJoin (74) + : :- CometProject (72) + : : +- CometBroadcastHashJoin (71) + : : :- CometBroadcastHashJoin (69) + : : : :- CometFilter (67) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (66) + : : : +- ReusedExchange (68) + : : +- ReusedExchange (70) + : +- ReusedExchange (73) + +- CometProject (95) + +- CometFilter (94) + +- CometHashAggregate (93) + +- CometColumnarExchange (92) + +- CometHashAggregate (91) + +- CometProject (90) + +- CometBroadcastHashJoin (89) + :- CometProject (87) + : +- CometBroadcastHashJoin (86) + : :- CometBroadcastHashJoin (84) + : : :- CometFilter (82) + : : : +- CometScan parquet spark_catalog.default.web_sales (81) + : : +- ReusedExchange (83) + : +- ReusedExchange (85) + +- ReusedExchange (88) (1) Scan parquet spark_catalog.default.store_sales @@ -118,24 +115,18 @@ 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 : 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 +(3) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) -(6) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(7) Scan parquet spark_catalog.default.store_sales +(5) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] @@ -143,22 +134,22 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(8) CometFilter +(6) CometFilter Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(9) Scan parquet spark_catalog.default.item +(7) 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 -(10) CometFilter +(8) 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)) -(11) Scan parquet spark_catalog.default.catalog_sales +(9) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] @@ -166,116 +157,111 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(12) CometFilter +(10) CometFilter Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) -(13) Scan parquet spark_catalog.default.item +(11) 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 -(14) CometFilter +(12) 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) -(15) CometBroadcastExchange +(13) 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] -(16) CometBroadcastHashJoin +(14) 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 -(17) CometProject +(15) 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 +(16) 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 -(19) CometFilter +(17) 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 +(18) CometProject Input [2]: [d_date_sk#24, d_year#25] Arguments: [d_date_sk#24], [d_date_sk#24] -(21) CometBroadcastExchange +(19) CometBroadcastExchange Input [1]: [d_date_sk#24] Arguments: [d_date_sk#24] -(22) CometBroadcastHashJoin +(20) 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 +(21) 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] -(24) CometBroadcastExchange +(22) CometBroadcastExchange Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] -(25) CometBroadcastHashJoin +(23) 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 -(26) CometBroadcastExchange +(24) CometBroadcastExchange Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(27) CometBroadcastHashJoin +(25) 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 -(28) CometProject +(26) 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] -(29) ReusedExchange [Reuses operator id: 21] +(27) ReusedExchange [Reuses operator id: 19] Output [1]: [d_date_sk#26] -(30) CometBroadcastHashJoin +(28) 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 -(31) CometProject +(29) 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] -(32) CometHashAggregate +(30) CometHashAggregate Input [3]: [brand_id#27, class_id#28, category_id#29] Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] -(33) ColumnarToRow [codegen id : 1] +(31) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(34) Exchange -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 : 3] +(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#27, class_id#28, category_id#29] -(36) Scan parquet spark_catalog.default.web_sales +(33) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] @@ -283,506 +269,486 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(37) CometFilter +(34) CometFilter Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] Condition : isnotnull(ws_item_sk#30) -(38) ReusedExchange [Reuses operator id: 15] +(35) ReusedExchange [Reuses operator id: 13] Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -(39) CometBroadcastHashJoin +(36) 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) CometProject +(37) 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) ReusedExchange [Reuses operator id: 21] +(38) ReusedExchange [Reuses operator id: 19] Output [1]: [d_date_sk#37] -(42) CometBroadcastHashJoin +(39) 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) CometProject +(40) 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) ColumnarToRow [codegen id : 2] +(41) CometBroadcastExchange Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] -(45) BroadcastExchange -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 : 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 +(42) CometBroadcastHashJoin +Left output [3]: [brand_id#27, class_id#28, category_id#29] +Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [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)], [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)], LeftSemi, BuildRight -(47) BroadcastExchange +(43) CometBroadcastExchange 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] +Arguments: [brand_id#27, class_id#28, category_id#29] -(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#27, class_id#28, category_id#29] -Join type: Inner -Join condition: None +(44) CometBroadcastHashJoin +Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Right output [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight -(49) Project [codegen id : 4] -Output [1]: [i_item_sk#6 AS ss_item_sk#38] +(45) CometProject 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] +Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] -(50) BroadcastExchange +(46) CometBroadcastExchange Input [1]: [ss_item_sk#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: [ss_item_sk#38] -(51) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#38] -Join type: LeftSemi -Join condition: None +(47) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Right output [1]: [ss_item_sk#38] +Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight -(52) Scan parquet spark_catalog.default.item +(48) Scan parquet spark_catalog.default.item 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 +(49) CometFilter 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 : 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] +(50) ReusedExchange [Reuses operator id: 46] Output [1]: [ss_item_sk#38] -(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 +(51) CometBroadcastHashJoin +Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Right output [1]: [ss_item_sk#38] +Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight -(57) BroadcastExchange +(52) CometBroadcastExchange 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] +Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -(58) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#39] -Join type: Inner -Join condition: None +(53) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight -(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] +(54) CometProject 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] +Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -(60) ReusedExchange [Reuses operator id: 127] -Output [1]: [d_date_sk#43] +(55) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#43, d_year#44, d_moy#45] +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 + +(56) CometFilter +Input [3]: [d_date_sk#43, 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 = 11)) AND isnotnull(d_date_sk#43)) + +(57) CometProject +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Arguments: [d_date_sk#43], [d_date_sk#43] + +(58) CometBroadcastExchange +Input [1]: [d_date_sk#43] +Arguments: [d_date_sk#43] -(61) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#43] -Join type: Inner -Join condition: None +(59) CometBroadcastHashJoin +Left 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] +Right output [1]: [d_date_sk#43] +Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight -(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] +(60) CometProject 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] +Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -(63) HashAggregate [codegen id : 11] +(61) CometHashAggregate 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#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#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] +(62) CometColumnarExchange +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] +Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(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] +(63) CometHashAggregate +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] 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))#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 : 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)))) +(64) CometFilter +Input [5]: [i_brand_id#40, i_class_id#41, i_category_id#42, sales#49, number_sales#50] +Condition : (isnotnull(sales#49) AND (cast(sales#49 as decimal(32,6)) > cast(Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) -(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] +(65) CometProject +Input [5]: [i_brand_id#40, i_class_id#41, i_category_id#42, sales#49, number_sales#50] +Arguments: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56], [sales#49, number_sales#50, store AS channel#53, i_brand_id#40 AS i_brand_id#54, i_class_id#41 AS i_class_id#55, i_category_id#42 AS i_category_id#56] -(68) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#60, cs_quantity#61, cs_list_price#62, cs_sold_date_sk#63] +(66) Scan parquet spark_catalog.default.catalog_sales +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#63), dynamicpruningexpression(cs_sold_date_sk#63 IN dynamicpruning#64)] +PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_sold_date_sk#60 IN dynamicpruning#61)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(69) CometFilter -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 : 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#65] - -(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#66, i_brand_id#67, i_class_id#68, i_category_id#69] - -(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 : 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: 127] -Output [1]: [d_date_sk#70] - -(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 : 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 : 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#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 : 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 : 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 : 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#82, ws_quantity#83, ws_list_price#84, ws_sold_date_sk#85] +(67) CometFilter +Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] +Condition : isnotnull(cs_item_sk#57) + +(68) ReusedExchange [Reuses operator id: 46] +Output [1]: [ss_item_sk#62] + +(69) CometBroadcastHashJoin +Left output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] +Right output [1]: [ss_item_sk#62] +Arguments: [cs_item_sk#57], [ss_item_sk#62], LeftSemi, BuildRight + +(70) ReusedExchange [Reuses operator id: 52] +Output [4]: [i_item_sk#63, i_brand_id#64, i_class_id#65, i_category_id#66] + +(71) CometBroadcastHashJoin +Left output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] +Right output [4]: [i_item_sk#63, i_brand_id#64, i_class_id#65, i_category_id#66] +Arguments: [cs_item_sk#57], [i_item_sk#63], Inner, BuildRight + +(72) CometProject +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] +Arguments: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#64, i_class_id#65, i_category_id#66], [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#64, i_class_id#65, i_category_id#66] + +(73) ReusedExchange [Reuses operator id: 58] +Output [1]: [d_date_sk#67] + +(74) CometBroadcastHashJoin +Left 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] +Right output [1]: [d_date_sk#67] +Arguments: [cs_sold_date_sk#60], [d_date_sk#67], Inner, BuildRight + +(75) CometProject +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] +Arguments: [cs_quantity#58, cs_list_price#59, i_brand_id#64, i_class_id#65, i_category_id#66], [cs_quantity#58, cs_list_price#59, i_brand_id#64, i_class_id#65, i_category_id#66] + +(76) CometHashAggregate +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)] + +(77) CometColumnarExchange +Input [6]: [i_brand_id#64, i_class_id#65, i_category_id#66, sum#68, isEmpty#69, count#70] +Arguments: hashpartitioning(i_brand_id#64, i_class_id#65, i_category_id#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(78) CometHashAggregate +Input [6]: [i_brand_id#64, i_class_id#65, i_category_id#66, sum#68, isEmpty#69, count#70] +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)] + +(79) CometFilter +Input [5]: [i_brand_id#64, i_class_id#65, i_category_id#66, sales#71, number_sales#72] +Condition : (isnotnull(sales#71) AND (cast(sales#71 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) + +(80) CometProject +Input [5]: [i_brand_id#64, i_class_id#65, i_category_id#66, sales#71, number_sales#72] +Arguments: [sales#71, number_sales#72, channel#73, i_brand_id#64, i_class_id#65, i_category_id#66], [sales#71, number_sales#72, catalog AS channel#73, i_brand_id#64, i_class_id#65, i_category_id#66] + +(81) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#85), dynamicpruningexpression(ws_sold_date_sk#85 IN dynamicpruning#86)] +PartitionFilters: [isnotnull(ws_sold_date_sk#77), dynamicpruningexpression(ws_sold_date_sk#77 IN dynamicpruning#78)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(85) CometFilter -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 : 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#87] - -(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#88, i_brand_id#89, i_class_id#90, i_category_id#91] - -(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 : 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: 127] -Output [1]: [d_date_sk#92] - -(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 : 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 : 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#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 : 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 : 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 : 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 : 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 : 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#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 : 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#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] +(82) CometFilter +Input [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77] +Condition : isnotnull(ws_item_sk#74) + +(83) ReusedExchange [Reuses operator id: 46] +Output [1]: [ss_item_sk#79] + +(84) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77] +Right output [1]: [ss_item_sk#79] +Arguments: [ws_item_sk#74], [ss_item_sk#79], LeftSemi, BuildRight + +(85) ReusedExchange [Reuses operator id: 52] +Output [4]: [i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] + +(86) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77] +Right output [4]: [i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] +Arguments: [ws_item_sk#74], [i_item_sk#80], Inner, BuildRight + +(87) CometProject +Input [8]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] +Arguments: [ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_brand_id#81, i_class_id#82, i_category_id#83], [ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_brand_id#81, i_class_id#82, i_category_id#83] + +(88) ReusedExchange [Reuses operator id: 58] +Output [1]: [d_date_sk#84] + +(89) CometBroadcastHashJoin +Left output [6]: [ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_brand_id#81, i_class_id#82, i_category_id#83] +Right output [1]: [d_date_sk#84] +Arguments: [ws_sold_date_sk#77], [d_date_sk#84], Inner, BuildRight + +(90) CometProject +Input [7]: [ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_brand_id#81, i_class_id#82, i_category_id#83, d_date_sk#84] +Arguments: [ws_quantity#75, ws_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83], [ws_quantity#75, ws_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] + +(91) CometHashAggregate +Input [5]: [ws_quantity#75, ws_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] +Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] +Functions [2]: [partial_sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76)), partial_count(1)] + +(92) CometColumnarExchange +Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#85, isEmpty#86, count#87] +Arguments: hashpartitioning(i_brand_id#81, i_class_id#82, i_category_id#83, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(93) CometHashAggregate +Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#85, isEmpty#86, count#87] +Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] +Functions [2]: [sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76)), count(1)] + +(94) CometFilter +Input [5]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#88, number_sales#89] +Condition : (isnotnull(sales#88) AND (cast(sales#88 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) + +(95) CometProject +Input [5]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#88, number_sales#89] +Arguments: [sales#88, number_sales#89, channel#90, i_brand_id#81, i_class_id#82, i_category_id#83], [sales#88, number_sales#89, web AS channel#90, i_brand_id#81, i_class_id#82, i_category_id#83] + +(96) CometUnion +Child 0 Input [6]: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Child 1 Input [6]: [sales#71, number_sales#72, channel#73, i_brand_id#64, i_class_id#65, i_category_id#66] +Child 2 Input [6]: [sales#88, number_sales#89, channel#90, i_brand_id#81, i_class_id#82, i_category_id#83] + +(97) CometExpand +Input [6]: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Arguments: [[sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56, 0], [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, null, 1], [sales#49, number_sales#50, channel#53, i_brand_id#54, null, null, 3], [sales#49, number_sales#50, channel#53, null, null, null, 7], [sales#49, number_sales#50, null, null, null, null, 15]], [sales#49, number_sales#50, channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, spark_grouping_id#95] + +(98) CometHashAggregate +Input [7]: [sales#49, number_sales#50, channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, spark_grouping_id#95] +Keys [5]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, spark_grouping_id#95] +Functions [2]: [partial_sum(sales#49), partial_sum(number_sales#50)] + +(99) CometColumnarExchange +Input [8]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, spark_grouping_id#95, sum#96, isEmpty#97, sum#98] +Arguments: hashpartitioning(channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, spark_grouping_id#95, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(100) CometHashAggregate +Input [8]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, spark_grouping_id#95, sum#96, isEmpty#97, sum#98] +Keys [5]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, spark_grouping_id#95] +Functions [2]: [sum(sales#49), sum(number_sales#50)] + +(101) CometTakeOrderedAndProject +Input [6]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, sum(sales)#99, sum(number_sales)#100] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#91 ASC NULLS FIRST,i_brand_id#92 ASC NULLS FIRST,i_class_id#93 ASC NULLS FIRST,i_category_id#94 ASC NULLS FIRST], output=[channel#91,i_brand_id#92,i_class_id#93,i_category_id#94,sum(sales)#99,sum(number_sales)#100]), [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, sum(sales)#99, sum(number_sales)#100], 100, [channel#91 ASC NULLS FIRST, i_brand_id#92 ASC NULLS FIRST, i_class_id#93 ASC NULLS FIRST, i_category_id#94 ASC NULLS FIRST], [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, sum(sales)#99, sum(number_sales)#100] + +(102) ColumnarToRow [codegen id : 1] +Input [6]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, sum(sales)#99, sum(number_sales)#100] ===== Subqueries ===== -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#119, ss_list_price#120, ss_sold_date_sk#121] +Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#51, [id=#52] +* ColumnarToRow (119) ++- CometHashAggregate (118) + +- CometColumnarExchange (117) + +- CometHashAggregate (116) + +- CometUnion (115) + :- CometProject (106) + : +- CometBroadcastHashJoin (105) + : :- CometScan parquet spark_catalog.default.store_sales (103) + : +- ReusedExchange (104) + :- CometProject (110) + : +- CometBroadcastHashJoin (109) + : :- CometScan parquet spark_catalog.default.catalog_sales (107) + : +- ReusedExchange (108) + +- CometProject (114) + +- CometBroadcastHashJoin (113) + :- CometScan parquet spark_catalog.default.web_sales (111) + +- ReusedExchange (112) + + +(103) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_quantity#101, ss_list_price#102, ss_sold_date_sk#103] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#121), dynamicpruningexpression(ss_sold_date_sk#121 IN dynamicpruning#122)] +PartitionFilters: [isnotnull(ss_sold_date_sk#103), dynamicpruningexpression(ss_sold_date_sk#103 IN dynamicpruning#104)] ReadSchema: struct -(107) ReusedExchange [Reuses operator id: 21] -Output [1]: [d_date_sk#123] +(104) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#105] -(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 +(105) CometBroadcastHashJoin +Left output [3]: [ss_quantity#101, ss_list_price#102, ss_sold_date_sk#103] +Right output [1]: [d_date_sk#105] +Arguments: [ss_sold_date_sk#103], [d_date_sk#105], Inner, BuildRight -(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] +(106) CometProject +Input [4]: [ss_quantity#101, ss_list_price#102, ss_sold_date_sk#103, d_date_sk#105] +Arguments: [quantity#106, list_price#107], [ss_quantity#101 AS quantity#106, ss_list_price#102 AS list_price#107] -(110) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#126, cs_list_price#127, cs_sold_date_sk#128] +(107) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#108, cs_list_price#109, cs_sold_date_sk#110] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#128), dynamicpruningexpression(cs_sold_date_sk#128 IN dynamicpruning#129)] +PartitionFilters: [isnotnull(cs_sold_date_sk#110), dynamicpruningexpression(cs_sold_date_sk#110 IN dynamicpruning#111)] ReadSchema: struct -(111) ReusedExchange [Reuses operator id: 21] -Output [1]: [d_date_sk#130] +(108) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#112] -(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 +(109) CometBroadcastHashJoin +Left output [3]: [cs_quantity#108, cs_list_price#109, cs_sold_date_sk#110] +Right output [1]: [d_date_sk#112] +Arguments: [cs_sold_date_sk#110], [d_date_sk#112], Inner, BuildRight -(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] +(110) CometProject +Input [4]: [cs_quantity#108, cs_list_price#109, cs_sold_date_sk#110, d_date_sk#112] +Arguments: [quantity#113, list_price#114], [cs_quantity#108 AS quantity#113, cs_list_price#109 AS list_price#114] -(114) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#133, ws_list_price#134, ws_sold_date_sk#135] +(111) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#115, ws_list_price#116, ws_sold_date_sk#117] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#135), dynamicpruningexpression(ws_sold_date_sk#135 IN dynamicpruning#136)] +PartitionFilters: [isnotnull(ws_sold_date_sk#117), dynamicpruningexpression(ws_sold_date_sk#117 IN dynamicpruning#118)] ReadSchema: struct -(115) ReusedExchange [Reuses operator id: 21] -Output [1]: [d_date_sk#137] +(112) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#119] -(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 +(113) CometBroadcastHashJoin +Left output [3]: [ws_quantity#115, ws_list_price#116, ws_sold_date_sk#117] +Right output [1]: [d_date_sk#119] +Arguments: [ws_sold_date_sk#117], [d_date_sk#119], Inner, BuildRight -(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] +(114) CometProject +Input [4]: [ws_quantity#115, ws_list_price#116, ws_sold_date_sk#117, d_date_sk#119] +Arguments: [quantity#120, list_price#121], [ws_quantity#115 AS quantity#120, ws_list_price#116 AS list_price#121] -(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] +(115) CometUnion +Child 0 Input [2]: [quantity#106, list_price#107] +Child 1 Input [2]: [quantity#113, list_price#114] +Child 2 Input [2]: [quantity#120, list_price#121] -(119) CometHashAggregate -Input [2]: [quantity#124, list_price#125] +(116) CometHashAggregate +Input [2]: [quantity#106, list_price#107] Keys: [] -Functions [1]: [partial_avg((cast(quantity#124 as decimal(10,0)) * list_price#125))] +Functions [1]: [partial_avg((cast(quantity#106 as decimal(10,0)) * list_price#107))] -(120) ColumnarToRow [codegen id : 1] -Input [2]: [sum#140, count#141] +(117) CometColumnarExchange +Input [2]: [sum#122, count#123] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(121) Exchange -Input [2]: [sum#140, count#141] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] - -(122) HashAggregate [codegen id : 2] -Input [2]: [sum#140, count#141] +(118) CometHashAggregate +Input [2]: [sum#122, count#123] Keys: [] -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))#142] -Results [1]: [avg((cast(quantity#124 as decimal(10,0)) * list_price#125))#142 AS average_sales#143] +Functions [1]: [avg((cast(quantity#106 as decimal(10,0)) * list_price#107))] + +(119) ColumnarToRow [codegen id : 1] +Input [1]: [average_sales#124] -Subquery:2 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#121 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 103 Hosting Expression = ss_sold_date_sk#103 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#128 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 107 Hosting Expression = cs_sold_date_sk#110 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 114 Hosting Expression = ws_sold_date_sk#135 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 111 Hosting Expression = ws_sold_date_sk#117 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (127) -+- * ColumnarToRow (126) - +- CometProject (125) - +- CometFilter (124) - +- CometScan parquet spark_catalog.default.date_dim (123) +BroadcastExchange (124) ++- * ColumnarToRow (123) + +- CometProject (122) + +- CometFilter (121) + +- CometScan parquet spark_catalog.default.date_dim (120) -(123) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#43, d_year#144, d_moy#145] +(120) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#43, d_year#44, d_moy#45] 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 -(124) CometFilter -Input [3]: [d_date_sk#43, d_year#144, d_moy#145] -Condition : ((((isnotnull(d_year#144) AND isnotnull(d_moy#145)) AND (d_year#144 = 2001)) AND (d_moy#145 = 11)) AND isnotnull(d_date_sk#43)) +(121) CometFilter +Input [3]: [d_date_sk#43, 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 = 11)) AND isnotnull(d_date_sk#43)) -(125) CometProject -Input [3]: [d_date_sk#43, d_year#144, d_moy#145] +(122) CometProject +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] Arguments: [d_date_sk#43], [d_date_sk#43] -(126) ColumnarToRow [codegen id : 1] +(123) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] -(127) BroadcastExchange +(124) BroadcastExchange Input [1]: [d_date_sk#43] -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=7] -Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (132) -+- * ColumnarToRow (131) - +- CometProject (130) - +- CometFilter (129) - +- CometScan parquet spark_catalog.default.date_dim (128) +Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 +BroadcastExchange (129) ++- * ColumnarToRow (128) + +- CometProject (127) + +- CometFilter (126) + +- CometScan parquet spark_catalog.default.date_dim (125) -(128) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#146] +(125) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_year#125] 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 -(129) CometFilter -Input [2]: [d_date_sk#26, d_year#146] -Condition : (((isnotnull(d_year#146) AND (d_year#146 >= 1999)) AND (d_year#146 <= 2001)) AND isnotnull(d_date_sk#26)) +(126) CometFilter +Input [2]: [d_date_sk#26, d_year#125] +Condition : (((isnotnull(d_year#125) AND (d_year#125 >= 1999)) AND (d_year#125 <= 2001)) AND isnotnull(d_date_sk#26)) -(130) CometProject -Input [2]: [d_date_sk#26, d_year#146] +(127) CometProject +Input [2]: [d_date_sk#26, d_year#125] Arguments: [d_date_sk#26], [d_date_sk#26] -(131) ColumnarToRow [codegen id : 1] +(128) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] -(132) BroadcastExchange +(129) BroadcastExchange Input [1]: [d_date_sk#26] -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=8] -Subquery:7 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 +Subquery:7 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 82 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] +Subquery:9 Hosting operator id = 79 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] -Subquery:10 Hosting operator id = 68 Hosting Expression = cs_sold_date_sk#63 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 66 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 98 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] +Subquery:11 Hosting operator id = 94 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] -Subquery:12 Hosting operator id = 84 Hosting Expression = ws_sold_date_sk#85 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 81 Hosting Expression = ws_sold_date_sk#77 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/simplified.txt index 2257d398f..bb69fc92a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/simplified.txt @@ -1,190 +1,149 @@ -TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] - 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 (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 (12) - Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] - Filter [sales] - Subquery #3 - WholeStageCodegen (2) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - InputAdapter - Exchange #14 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [sum,count,quantity,list_price] - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,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_quantity,cs_list_price,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_quantity,ws_list_price,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] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales),spark_grouping_id,sum,isEmpty,sum,sum(sales),sum(number_salesL)] + CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum,sales,number_sales] + CometExpand [channel,i_brand_id,i_class_id,i_category_id] [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] + CometUnion [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] + CometProject [i_brand_id,i_class_id,i_category_id] [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] + CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales] + Subquery #3 + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #2 - 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] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,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 #5 - 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 + CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] + CometColumnarExchange #15 + CometHashAggregate [sum,count,quantity,list_price] + CometUnion [quantity,list_price] + CometProject [ss_quantity,ss_list_price] [quantity,list_price] + CometBroadcastHashJoin [ss_quantity,ss_list_price,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_quantity,cs_list_price,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_quantity,ws_list_price,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 + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #2 + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ss_quantity,ss_list_price] + CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk] #4 + CometProject [i_item_sk] [ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,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 [brand_id,class_id,category_id] #5 + CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + CometHashAggregate [brand_id,class_id,category_id] + CometColumnarExchange [brand_id,class_id,category_id] #6 + 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,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 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,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8 - CometBroadcastHashJoin [i_item_sk,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 [i_brand_id,i_class_id,i_category_id] #9 - CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [cs_item_sk,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 - 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 [d_date_sk] #11 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #11 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ws_item_sk,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 - ReusedExchange [d_date_sk] #11 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (9) - BroadcastHashJoin [i_item_sk,ss_item_sk] - 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 - ReusedExchange [ss_item_sk] #4 - InputAdapter - ReusedExchange [d_date_sk] #3 - 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] #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] - Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - BroadcastHashJoin [cs_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - InputAdapter - ReusedExchange [d_date_sk] #3 - 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] #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] - Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - BroadcastHashJoin [ws_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - InputAdapter - ReusedExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8 + CometBroadcastHashJoin [i_item_sk,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 [i_brand_id,i_class_id,i_category_id] #9 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + 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 [d_date_sk] #11 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #11 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #12 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + ReusedExchange [d_date_sk] #11 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] + 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] + ReusedExchange [ss_item_sk] #4 + CometBroadcastExchange [d_date_sk] #14 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] + CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales] + ReusedSubquery [average_sales] #3 + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1)] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #16 + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,cs_quantity,cs_list_price] + CometProject [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,ss_item_sk] + CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [ss_item_sk] #4 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + ReusedExchange [d_date_sk] #14 + CometProject [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] + CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales] + ReusedSubquery [average_sales] #3 + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1)] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #17 + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ws_quantity,ws_list_price] + CometProject [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,ss_item_sk] + CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [ss_item_sk] #4 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + ReusedExchange [d_date_sk] #14 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/explain.txt index 4270241a8..7bb99e447 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/explain.txt @@ -1,88 +1,89 @@ == Physical Plan == -TakeOrderedAndProject (84) -+- * BroadcastHashJoin Inner BuildRight (83) - :- * Filter (66) - : +- * HashAggregate (65) - : +- Exchange (64) - : +- * HashAggregate (63) - : +- * Project (62) - : +- * BroadcastHashJoin Inner BuildRight (61) - : :- * Project (59) - : : +- * BroadcastHashJoin Inner BuildRight (58) - : : :- * BroadcastHashJoin LeftSemi BuildRight (51) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (50) - : : : +- * Project (49) - : : : +- * BroadcastHashJoin Inner BuildRight (48) - : : : :- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.item (4) - : : : +- BroadcastExchange (47) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) - : : : :- * HashAggregate (35) - : : : : +- Exchange (34) - : : : : +- * 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) - : : : +- * 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) - : : : +- CometFilter (53) - : : : +- CometScan parquet spark_catalog.default.item (52) - : : +- ReusedExchange (55) - : +- ReusedExchange (60) - +- BroadcastExchange (82) - +- * Filter (81) - +- * HashAggregate (80) - +- Exchange (79) - +- * HashAggregate (78) - +- * Project (77) - +- * BroadcastHashJoin Inner BuildRight (76) - :- * Project (74) - : +- * BroadcastHashJoin Inner BuildRight (73) - : :- * BroadcastHashJoin LeftSemi BuildRight (71) - : : :- * ColumnarToRow (69) - : : : +- CometFilter (68) - : : : +- CometScan parquet spark_catalog.default.store_sales (67) - : : +- ReusedExchange (70) - : +- ReusedExchange (72) - +- ReusedExchange (75) +* ColumnarToRow (85) ++- CometTakeOrderedAndProject (84) + +- CometBroadcastHashJoin (83) + :- CometFilter (64) + : +- CometHashAggregate (63) + : +- CometColumnarExchange (62) + : +- CometHashAggregate (61) + : +- CometProject (60) + : +- CometBroadcastHashJoin (59) + : :- CometProject (54) + : : +- CometBroadcastHashJoin (53) + : : :- CometBroadcastHashJoin (47) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (46) + : : : +- CometProject (45) + : : : +- CometBroadcastHashJoin (44) + : : : :- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.item (3) + : : : +- CometBroadcastExchange (43) + : : : +- CometBroadcastHashJoin (42) + : : : :- CometHashAggregate (32) + : : : : +- CometColumnarExchange (31) + : : : : +- CometHashAggregate (30) + : : : : +- CometProject (29) + : : : : +- CometBroadcastHashJoin (28) + : : : : :- CometProject (26) + : : : : : +- CometBroadcastHashJoin (25) + : : : : : :- CometFilter (6) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (5) + : : : : : +- CometBroadcastExchange (24) + : : : : : +- CometBroadcastHashJoin (23) + : : : : : :- CometFilter (8) + : : : : : : +- CometScan parquet spark_catalog.default.item (7) + : : : : : +- CometBroadcastExchange (22) + : : : : : +- CometProject (21) + : : : : : +- CometBroadcastHashJoin (20) + : : : : : :- CometProject (15) + : : : : : : +- CometBroadcastHashJoin (14) + : : : : : : :- CometFilter (10) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (9) + : : : : : : +- CometBroadcastExchange (13) + : : : : : : +- CometFilter (12) + : : : : : : +- CometScan parquet spark_catalog.default.item (11) + : : : : : +- CometBroadcastExchange (19) + : : : : : +- CometProject (18) + : : : : : +- CometFilter (17) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (16) + : : : : +- ReusedExchange (27) + : : : +- CometBroadcastExchange (41) + : : : +- CometProject (40) + : : : +- CometBroadcastHashJoin (39) + : : : :- CometProject (37) + : : : : +- CometBroadcastHashJoin (36) + : : : : :- CometFilter (34) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (33) + : : : : +- ReusedExchange (35) + : : : +- ReusedExchange (38) + : : +- CometBroadcastExchange (52) + : : +- CometBroadcastHashJoin (51) + : : :- CometFilter (49) + : : : +- CometScan parquet spark_catalog.default.item (48) + : : +- ReusedExchange (50) + : +- CometBroadcastExchange (58) + : +- CometProject (57) + : +- CometFilter (56) + : +- CometScan parquet spark_catalog.default.date_dim (55) + +- CometBroadcastExchange (82) + +- CometFilter (81) + +- CometHashAggregate (80) + +- CometColumnarExchange (79) + +- CometHashAggregate (78) + +- CometProject (77) + +- CometBroadcastHashJoin (76) + :- CometProject (71) + : +- CometBroadcastHashJoin (70) + : :- CometBroadcastHashJoin (68) + : : :- CometFilter (66) + : : : +- CometScan parquet spark_catalog.default.store_sales (65) + : : +- ReusedExchange (67) + : +- ReusedExchange (69) + +- CometBroadcastExchange (75) + +- CometProject (74) + +- CometFilter (73) + +- CometScan parquet spark_catalog.default.date_dim (72) (1) Scan parquet spark_catalog.default.store_sales @@ -97,24 +98,18 @@ 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 : 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 +(3) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) -(6) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(7) Scan parquet spark_catalog.default.store_sales +(5) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] @@ -122,22 +117,22 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(8) CometFilter +(6) CometFilter Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(9) Scan parquet spark_catalog.default.item +(7) 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 -(10) CometFilter +(8) 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)) -(11) Scan parquet spark_catalog.default.catalog_sales +(9) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] @@ -145,116 +140,111 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(12) CometFilter +(10) CometFilter Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) -(13) Scan parquet spark_catalog.default.item +(11) 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 -(14) CometFilter +(12) 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) -(15) CometBroadcastExchange +(13) 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] -(16) CometBroadcastHashJoin +(14) 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 -(17) CometProject +(15) 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 +(16) 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 -(19) CometFilter +(17) 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 +(18) CometProject Input [2]: [d_date_sk#24, d_year#25] Arguments: [d_date_sk#24], [d_date_sk#24] -(21) CometBroadcastExchange +(19) CometBroadcastExchange Input [1]: [d_date_sk#24] Arguments: [d_date_sk#24] -(22) CometBroadcastHashJoin +(20) 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 +(21) 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] -(24) CometBroadcastExchange +(22) CometBroadcastExchange Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] -(25) CometBroadcastHashJoin +(23) 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 -(26) CometBroadcastExchange +(24) CometBroadcastExchange Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(27) CometBroadcastHashJoin +(25) 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 -(28) CometProject +(26) 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] -(29) ReusedExchange [Reuses operator id: 21] +(27) ReusedExchange [Reuses operator id: 19] Output [1]: [d_date_sk#26] -(30) CometBroadcastHashJoin +(28) 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 -(31) CometProject +(29) 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] -(32) CometHashAggregate +(30) CometHashAggregate Input [3]: [brand_id#27, class_id#28, category_id#29] Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] -(33) ColumnarToRow [codegen id : 1] -Input [3]: [brand_id#27, class_id#28, category_id#29] - -(34) Exchange +(31) CometColumnarExchange 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] +Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(35) HashAggregate [codegen id : 3] +(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#27, class_id#28, category_id#29] -(36) Scan parquet spark_catalog.default.web_sales +(33) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] @@ -262,482 +252,492 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(37) CometFilter +(34) CometFilter Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] Condition : isnotnull(ws_item_sk#30) -(38) ReusedExchange [Reuses operator id: 15] +(35) ReusedExchange [Reuses operator id: 13] Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -(39) CometBroadcastHashJoin +(36) 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) CometProject +(37) 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) ReusedExchange [Reuses operator id: 21] +(38) ReusedExchange [Reuses operator id: 19] Output [1]: [d_date_sk#37] -(42) CometBroadcastHashJoin +(39) 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) CometProject +(40) 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) ColumnarToRow [codegen id : 2] -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(45) BroadcastExchange +(41) CometBroadcastExchange 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] +Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] -(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 +(42) CometBroadcastHashJoin +Left output [3]: [brand_id#27, class_id#28, category_id#29] +Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [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)], [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)], LeftSemi, BuildRight -(47) BroadcastExchange +(43) CometBroadcastExchange 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] +Arguments: [brand_id#27, class_id#28, category_id#29] -(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#27, class_id#28, category_id#29] -Join type: Inner -Join condition: None +(44) CometBroadcastHashJoin +Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Right output [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight -(49) Project [codegen id : 4] -Output [1]: [i_item_sk#6 AS ss_item_sk#38] +(45) CometProject 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] +Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] -(50) BroadcastExchange +(46) CometBroadcastExchange Input [1]: [ss_item_sk#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: [ss_item_sk#38] -(51) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#38] -Join type: LeftSemi -Join condition: None +(47) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Right output [1]: [ss_item_sk#38] +Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight -(52) Scan parquet spark_catalog.default.item +(48) Scan parquet spark_catalog.default.item 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 +(49) CometFilter 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 : 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] +(50) ReusedExchange [Reuses operator id: 46] Output [1]: [ss_item_sk#38] -(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 +(51) CometBroadcastHashJoin +Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Right output [1]: [ss_item_sk#38] +Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight -(57) BroadcastExchange +(52) CometBroadcastExchange 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] +Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -(58) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#39] -Join type: Inner -Join condition: None +(53) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight -(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] +(54) CometProject 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] +Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] + +(55) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#43, d_week_seq#44] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] +ReadSchema: struct + +(56) CometFilter +Input [2]: [d_date_sk#43, d_week_seq#44] +Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = ReusedSubquery Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) + +(57) CometProject +Input [2]: [d_date_sk#43, d_week_seq#44] +Arguments: [d_date_sk#43], [d_date_sk#43] -(60) ReusedExchange [Reuses operator id: 106] -Output [1]: [d_date_sk#43] +(58) CometBroadcastExchange +Input [1]: [d_date_sk#43] +Arguments: [d_date_sk#43] -(61) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#43] -Join type: Inner -Join condition: None +(59) CometBroadcastHashJoin +Left 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] +Right output [1]: [d_date_sk#43] +Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight -(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] +(60) CometProject 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] +Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -(63) HashAggregate [codegen id : 11] +(61) CometHashAggregate 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#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 +(62) CometColumnarExchange 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] +Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(65) HashAggregate [codegen id : 24] +(63) CometHashAggregate 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))#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 : 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)))) +(64) CometFilter +Input [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, 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)))) -(67) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60] +(65) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#60), dynamicpruningexpression(ss_sold_date_sk#60 IN dynamicpruning#61)] +PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(68) CometFilter -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 : 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#62] - -(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#63, i_brand_id#64, i_class_id#65, i_category_id#66] - -(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 : 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: 120] -Output [1]: [d_date_sk#67] - -(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 : 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 : 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#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 : 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 : 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#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 : 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#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] +(66) CometFilter +Input [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] +Condition : isnotnull(ss_item_sk#55) + +(67) ReusedExchange [Reuses operator id: 46] +Output [1]: [ss_item_sk#60] + +(68) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] +Right output [1]: [ss_item_sk#60] +Arguments: [ss_item_sk#55], [ss_item_sk#60], LeftSemi, BuildRight + +(69) ReusedExchange [Reuses operator id: 52] +Output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] + +(70) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] +Right output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] +Arguments: [ss_item_sk#55], [i_item_sk#61], Inner, BuildRight + +(71) CometProject +Input [8]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] +Arguments: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64], [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64] + +(72) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#65, d_week_seq#66] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] +ReadSchema: struct + +(73) CometFilter +Input [2]: [d_date_sk#65, d_week_seq#66] +Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = ReusedSubquery Subquery scalar-subquery#67, [id=#68])) AND isnotnull(d_date_sk#65)) + +(74) CometProject +Input [2]: [d_date_sk#65, d_week_seq#66] +Arguments: [d_date_sk#65], [d_date_sk#65] + +(75) CometBroadcastExchange +Input [1]: [d_date_sk#65] +Arguments: [d_date_sk#65] + +(76) CometBroadcastHashJoin +Left output [6]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64] +Right output [1]: [d_date_sk#65] +Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight + +(77) CometProject +Input [7]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64, d_date_sk#65] +Arguments: [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64], [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64] + +(78) CometHashAggregate +Input [5]: [ss_quantity#56, ss_list_price#57, 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#56 as decimal(10,0)) * ss_list_price#57)), partial_count(1)] + +(79) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=3] + +(80) CometHashAggregate +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#56 as decimal(10,0)) * ss_list_price#57)), count(1)] + +(81) CometFilter +Input [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] +Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) + +(82) CometBroadcastExchange +Input [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] +Arguments: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] + +(83) CometBroadcastHashJoin +Left output [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] +Right output [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] +Arguments: [i_brand_id#40, i_class_id#41, i_category_id#42], [i_brand_id#62, i_class_id#63, i_category_id#64], Inner, BuildRight + +(84) CometTakeOrderedAndProject +Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#50,i_brand_id#40,i_class_id#41,i_category_id#42,sales#51,number_sales#52,channel#72,i_brand_id#62,i_class_id#63,i_category_id#64,sales#73,number_sales#74]), [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74], 100, [i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] + +(85) ColumnarToRow [codegen id : 1] +Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] ===== Subqueries ===== -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#79, ss_list_price#80, ss_sold_date_sk#81] +Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#53, [id=#54] +* ColumnarToRow (102) ++- CometHashAggregate (101) + +- CometColumnarExchange (100) + +- CometHashAggregate (99) + +- CometUnion (98) + :- CometProject (89) + : +- CometBroadcastHashJoin (88) + : :- CometScan parquet spark_catalog.default.store_sales (86) + : +- ReusedExchange (87) + :- CometProject (93) + : +- CometBroadcastHashJoin (92) + : :- CometScan parquet spark_catalog.default.catalog_sales (90) + : +- ReusedExchange (91) + +- CometProject (97) + +- CometBroadcastHashJoin (96) + :- CometScan parquet spark_catalog.default.web_sales (94) + +- ReusedExchange (95) + + +(86) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#81), dynamicpruningexpression(ss_sold_date_sk#81 IN dynamicpruning#82)] +PartitionFilters: [isnotnull(ss_sold_date_sk#77), dynamicpruningexpression(ss_sold_date_sk#77 IN dynamicpruning#78)] ReadSchema: struct -(86) ReusedExchange [Reuses operator id: 21] -Output [1]: [d_date_sk#83] +(87) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#79] -(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 +(88) CometBroadcastHashJoin +Left output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] +Right output [1]: [d_date_sk#79] +Arguments: [ss_sold_date_sk#77], [d_date_sk#79], Inner, BuildRight -(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] +(89) CometProject +Input [4]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77, d_date_sk#79] +Arguments: [quantity#80, list_price#81], [ss_quantity#75 AS quantity#80, ss_list_price#76 AS list_price#81] -(89) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#86, cs_list_price#87, cs_sold_date_sk#88] +(90) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#88), dynamicpruningexpression(cs_sold_date_sk#88 IN dynamicpruning#89)] +PartitionFilters: [isnotnull(cs_sold_date_sk#84), dynamicpruningexpression(cs_sold_date_sk#84 IN dynamicpruning#85)] ReadSchema: struct -(90) ReusedExchange [Reuses operator id: 21] -Output [1]: [d_date_sk#90] +(91) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#86] -(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 +(92) CometBroadcastHashJoin +Left output [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] +Right output [1]: [d_date_sk#86] +Arguments: [cs_sold_date_sk#84], [d_date_sk#86], Inner, BuildRight -(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] +(93) CometProject +Input [4]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84, d_date_sk#86] +Arguments: [quantity#87, list_price#88], [cs_quantity#82 AS quantity#87, cs_list_price#83 AS list_price#88] -(93) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#93, ws_list_price#94, ws_sold_date_sk#95] +(94) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#95), dynamicpruningexpression(ws_sold_date_sk#95 IN dynamicpruning#96)] +PartitionFilters: [isnotnull(ws_sold_date_sk#91), dynamicpruningexpression(ws_sold_date_sk#91 IN dynamicpruning#92)] ReadSchema: struct -(94) ReusedExchange [Reuses operator id: 21] -Output [1]: [d_date_sk#97] +(95) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#93] -(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 +(96) CometBroadcastHashJoin +Left output [3]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91] +Right output [1]: [d_date_sk#93] +Arguments: [ws_sold_date_sk#91], [d_date_sk#93], Inner, BuildRight -(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] +(97) CometProject +Input [4]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91, d_date_sk#93] +Arguments: [quantity#94, list_price#95], [ws_quantity#89 AS quantity#94, ws_list_price#90 AS list_price#95] -(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] +(98) CometUnion +Child 0 Input [2]: [quantity#80, list_price#81] +Child 1 Input [2]: [quantity#87, list_price#88] +Child 2 Input [2]: [quantity#94, list_price#95] -(98) CometHashAggregate -Input [2]: [quantity#84, list_price#85] +(99) CometHashAggregate +Input [2]: [quantity#80, list_price#81] Keys: [] -Functions [1]: [partial_avg((cast(quantity#84 as decimal(10,0)) * list_price#85))] +Functions [1]: [partial_avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] -(99) ColumnarToRow [codegen id : 1] -Input [2]: [sum#100, count#101] +(100) CometColumnarExchange +Input [2]: [sum#96, count#97] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(100) Exchange -Input [2]: [sum#100, count#101] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] - -(101) HashAggregate [codegen id : 2] -Input [2]: [sum#100, count#101] +(101) CometHashAggregate +Input [2]: [sum#96, count#97] Keys: [] -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))#102] -Results [1]: [avg((cast(quantity#84 as decimal(10,0)) * list_price#85))#102 AS average_sales#103] +Functions [1]: [avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] + +(102) ColumnarToRow [codegen id : 1] +Input [1]: [average_sales#98] -Subquery:2 Hosting operator id = 85 Hosting Expression = ss_sold_date_sk#81 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 86 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 89 Hosting Expression = cs_sold_date_sk#88 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#84 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 93 Hosting Expression = ws_sold_date_sk#95 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#91 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (106) -+- * ColumnarToRow (105) - +- CometProject (104) - +- CometFilter (103) - +- CometScan parquet spark_catalog.default.date_dim (102) +BroadcastExchange (107) ++- * ColumnarToRow (106) + +- CometProject (105) + +- CometFilter (104) + +- CometScan parquet spark_catalog.default.date_dim (103) -(102) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_week_seq#104] +(103) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#43, d_week_seq#44] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(103) CometFilter -Input [2]: [d_date_sk#43, 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#43)) +(104) CometFilter +Input [2]: [d_date_sk#43, d_week_seq#44] +Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) -(104) CometProject -Input [2]: [d_date_sk#43, d_week_seq#104] +(105) CometProject +Input [2]: [d_date_sk#43, d_week_seq#44] Arguments: [d_date_sk#43], [d_date_sk#43] -(105) ColumnarToRow [codegen id : 1] +(106) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] -(106) BroadcastExchange +(107) BroadcastExchange Input [1]: [d_date_sk#43] -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=5] -Subquery:6 Hosting operator id = 103 Hosting Expression = Subquery scalar-subquery#105, [id=#106] -* ColumnarToRow (110) -+- CometProject (109) - +- CometFilter (108) - +- CometScan parquet spark_catalog.default.date_dim (107) +Subquery:6 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] +* ColumnarToRow (111) ++- CometProject (110) + +- CometFilter (109) + +- CometScan parquet spark_catalog.default.date_dim (108) -(107) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#107, d_year#108, d_moy#109, d_dom#110] +(108) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] 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 -(108) 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)) +(109) CometFilter +Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] +Condition : (((((isnotnull(d_year#100) AND isnotnull(d_moy#101)) AND isnotnull(d_dom#102)) AND (d_year#100 = 2000)) AND (d_moy#101 = 12)) AND (d_dom#102 = 11)) -(109) 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] +(110) CometProject +Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] +Arguments: [d_week_seq#99], [d_week_seq#99] -(110) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#107] +(111) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#99] -Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (115) -+- * ColumnarToRow (114) - +- CometProject (113) - +- CometFilter (112) - +- CometScan parquet spark_catalog.default.date_dim (111) +Subquery:7 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 +BroadcastExchange (116) ++- * ColumnarToRow (115) + +- CometProject (114) + +- CometFilter (113) + +- CometScan parquet spark_catalog.default.date_dim (112) -(111) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#111] +(112) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_year#103] 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 -(112) CometFilter -Input [2]: [d_date_sk#26, d_year#111] -Condition : (((isnotnull(d_year#111) AND (d_year#111 >= 1999)) AND (d_year#111 <= 2001)) AND isnotnull(d_date_sk#26)) +(113) CometFilter +Input [2]: [d_date_sk#26, d_year#103] +Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1999)) AND (d_year#103 <= 2001)) AND isnotnull(d_date_sk#26)) -(113) CometProject -Input [2]: [d_date_sk#26, d_year#111] +(114) CometProject +Input [2]: [d_date_sk#26, d_year#103] Arguments: [d_date_sk#26], [d_date_sk#26] -(114) ColumnarToRow [codegen id : 1] +(115) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] -(115) BroadcastExchange +(116) BroadcastExchange Input [1]: [d_date_sk#26] -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=6] -Subquery:8 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 +Subquery:9 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 -Subquery:10 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#55, [id=#56] +Subquery:10 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -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) +Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#53, [id=#54] +Subquery:12 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 +BroadcastExchange (121) ++- * ColumnarToRow (120) + +- CometProject (119) + +- CometFilter (118) + +- CometScan parquet spark_catalog.default.date_dim (117) -(116) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#67, d_week_seq#112] + +(117) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#65, d_week_seq#66] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(117) CometFilter -Input [2]: [d_date_sk#67, 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#67)) +(118) CometFilter +Input [2]: [d_date_sk#65, d_week_seq#66] +Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = Subquery scalar-subquery#67, [id=#68])) AND isnotnull(d_date_sk#65)) -(118) CometProject -Input [2]: [d_date_sk#67, d_week_seq#112] -Arguments: [d_date_sk#67], [d_date_sk#67] +(119) CometProject +Input [2]: [d_date_sk#65, d_week_seq#66] +Arguments: [d_date_sk#65], [d_date_sk#65] -(119) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#67] +(120) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#65] -(120) BroadcastExchange -Input [1]: [d_date_sk#67] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] +(121) BroadcastExchange +Input [1]: [d_date_sk#65] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:12 Hosting operator id = 117 Hosting Expression = Subquery scalar-subquery#113, [id=#114] -* ColumnarToRow (124) -+- CometProject (123) - +- CometFilter (122) - +- CometScan parquet spark_catalog.default.date_dim (121) +Subquery:13 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#67, [id=#68] +* ColumnarToRow (125) ++- CometProject (124) + +- CometFilter (123) + +- CometScan parquet spark_catalog.default.date_dim (122) -(121) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#115, d_year#116, d_moy#117, d_dom#118] +(122) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] 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 -(122) 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)) +(123) CometFilter +Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] +Condition : (((((isnotnull(d_year#105) AND isnotnull(d_moy#106)) AND isnotnull(d_dom#107)) AND (d_year#105 = 1999)) AND (d_moy#106 = 12)) AND (d_dom#107 = 11)) + +(124) CometProject +Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] +Arguments: [d_week_seq#104], [d_week_seq#104] -(123) 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] +(125) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#104] -(124) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#115] +Subquery:14 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/simplified.txt index aa49638c7..a51d1d007 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/simplified.txt @@ -1,15 +1,15 @@ -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 (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 (2) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - InputAdapter - Exchange #13 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + CometBroadcastHashJoin [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Subquery #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] + CometColumnarExchange #14 CometHashAggregate [sum,count,quantity,list_price] CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] @@ -27,152 +27,127 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ 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 (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] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (4) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #1 + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ss_quantity,ss_list_price] + CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) 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 #4 - 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 (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,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 - CometBroadcastHashJoin [i_item_sk,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 [i_brand_id,i_class_id,i_category_id] #8 - CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [cs_item_sk,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - 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 [d_date_sk] #10 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #10 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (2) + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometBroadcastExchange [ss_item_sk] #3 + CometProject [i_item_sk] [ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,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 [brand_id,class_id,category_id] #4 + CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + CometHashAggregate [brand_id,class_id,category_id] + CometColumnarExchange [brand_id,class_id,category_id] #5 + 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,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 + CometBroadcastHashJoin [i_item_sk,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 [i_brand_id,i_class_id,i_category_id] #8 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + 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 [d_date_sk] #10 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #10 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #11 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + ReusedExchange [d_date_sk] #10 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] + 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] + ReusedExchange [ss_item_sk] #3 + CometBroadcastExchange [d_date_sk] #13 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #2 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometBroadcastExchange [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] #15 + CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + ReusedSubquery [average_sales] #4 + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #16 + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ss_quantity,ss_list_price] + CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_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 #17 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #6 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ws_item_sk,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - ReusedExchange [d_date_sk] #10 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - BroadcastHashJoin [i_item_sk,ss_item_sk] - 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 - ReusedExchange [ss_item_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - 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] #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] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_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 #16 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - ReusedExchange [ss_item_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - InputAdapter - ReusedExchange [d_date_sk] #16 + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + ReusedExchange [ss_item_sk] #3 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + CometBroadcastExchange [d_date_sk] #18 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #6 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/explain.txt index c4772ea80..af9c0909f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (23) -+- * HashAggregate (22) - +- Exchange (21) - +- * ColumnarToRow (20) +* ColumnarToRow (23) ++- CometTakeOrderedAndProject (22) + +- CometHashAggregate (21) + +- CometColumnarExchange (20) +- CometHashAggregate (19) +- CometProject (18) +- CometBroadcastHashJoin (17) @@ -117,23 +117,21 @@ Input [2]: [cs_sales_price#2, ca_zip#9] Keys [1]: [ca_zip#9] Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#2))] -(20) ColumnarToRow [codegen id : 1] +(20) CometColumnarExchange Input [2]: [ca_zip#9, sum#13] +Arguments: hashpartitioning(ca_zip#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(21) Exchange -Input [2]: [ca_zip#9, sum#13] -Arguments: hashpartitioning(ca_zip#9, 5), ENSURE_REQUIREMENTS, [plan_id=1] - -(22) HashAggregate [codegen id : 2] +(21) CometHashAggregate 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))#14] -Results [2]: [ca_zip#9, MakeDecimal(sum(UnscaledValue(cs_sales_price#2))#14,17,2) AS sum(cs_sales_price)#15] -(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] +(22) CometTakeOrderedAndProject +Input [2]: [ca_zip#9, sum(cs_sales_price)#14] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_zip#9 ASC NULLS FIRST], output=[ca_zip#9,sum(cs_sales_price)#14]), [ca_zip#9, sum(cs_sales_price)#14], 100, [ca_zip#9 ASC NULLS FIRST], [ca_zip#9, sum(cs_sales_price)#14] + +(23) ColumnarToRow [codegen id : 1] +Input [2]: [ca_zip#9, sum(cs_sales_price)#14] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/simplified.txt index b876fe4de..989ea0856 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/simplified.txt @@ -1,35 +1,33 @@ -TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] - WholeStageCodegen (2) - HashAggregate [ca_zip,sum] [sum(UnscaledValue(cs_sales_price)),sum(cs_sales_price),sum] - InputAdapter - Exchange [ca_zip] #1 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ca_zip,sum,cs_sales_price] - CometProject [cs_sales_price,ca_zip] - CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,ca_zip,d_date_sk] - CometProject [cs_sales_price,cs_sold_date_sk,ca_zip] - CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_state,ca_zip] - CometProject [cs_sales_price,cs_sold_date_sk,c_current_addr_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk,c_customer_sk,c_current_addr_sk] - CometFilter [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_state,ca_zip] #4 - CometFilter [ca_address_sk,ca_state,ca_zip] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [ca_zip,sum(cs_sales_price)] + CometHashAggregate [ca_zip,sum(cs_sales_price),sum,sum(UnscaledValue(cs_sales_price))] + CometColumnarExchange [ca_zip] #1 + CometHashAggregate [ca_zip,sum,cs_sales_price] + CometProject [cs_sales_price,ca_zip] + CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,ca_zip,d_date_sk] + CometProject [cs_sales_price,cs_sold_date_sk,ca_zip] + CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_state,ca_zip] + CometProject [cs_sales_price,cs_sold_date_sk,c_current_addr_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk,c_customer_sk,c_current_addr_sk] + CometFilter [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange [ca_address_sk,ca_state,ca_zip] #4 + CometFilter [ca_address_sk,ca_state,ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + 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-spark3_5/q16/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/explain.txt index ccec341ad..ebff95d06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/explain.txt @@ -1,49 +1,51 @@ == Physical Plan == -* HashAggregate (45) -+- Exchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * SortMergeJoin LeftAnti (19) - : : : :- * Project (13) - : : : : +- * SortMergeJoin LeftSemi (12) - : : : : :- * Sort (6) - : : : : : +- Exchange (5) - : : : : : +- * ColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : +- * Sort (11) - : : : : +- Exchange (10) - : : : : +- * ColumnarToRow (9) - : : : : +- CometProject (8) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (7) - : : : +- * Sort (18) - : : : +- Exchange (17) - : : : +- * ColumnarToRow (16) - : : : +- CometProject (15) - : : : +- CometScan parquet spark_catalog.default.catalog_returns (14) - : : +- BroadcastExchange (24) - : : +- * ColumnarToRow (23) - : : +- CometProject (22) - : : +- CometFilter (21) - : : +- CometScan parquet spark_catalog.default.date_dim (20) - : +- 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.call_center (34) +* HashAggregate (47) ++- * ColumnarToRow (46) + +- CometColumnarExchange (45) + +- RowToColumnar (44) + +- * HashAggregate (43) + +- * HashAggregate (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * SortMergeJoin LeftAnti (19) + : : : :- * Project (13) + : : : : +- * SortMergeJoin LeftSemi (12) + : : : : :- * ColumnarToRow (6) + : : : : : +- CometSort (5) + : : : : : +- CometColumnarExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : +- * ColumnarToRow (11) + : : : : +- CometSort (10) + : : : : +- CometColumnarExchange (9) + : : : : +- CometProject (8) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (7) + : : : +- * ColumnarToRow (18) + : : : +- CometSort (17) + : : : +- CometColumnarExchange (16) + : : : +- CometProject (15) + : : : +- CometScan parquet spark_catalog.default.catalog_returns (14) + : : +- BroadcastExchange (24) + : : +- * ColumnarToRow (23) + : : +- CometProject (22) + : : +- CometFilter (21) + : : +- CometScan parquet spark_catalog.default.date_dim (20) + : +- 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.call_center (34) (1) Scan parquet spark_catalog.default.catalog_sales @@ -61,16 +63,16 @@ Condition : ((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -(4) ColumnarToRow [codegen id : 1] +(4) CometColumnarExchange Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] +Arguments: hashpartitioning(cs_order_number#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(5) Exchange +(5) CometSort Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: hashpartitioning(cs_order_number#5, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_order_number#5 ASC NULLS FIRST] -(6) Sort [codegen id : 2] +(6) ColumnarToRow [codegen id : 1] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: [cs_order_number#5 ASC NULLS FIRST], false, 0 (7) Scan parquet spark_catalog.default.catalog_sales Output [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11] @@ -82,24 +84,24 @@ ReadSchema: struct Input [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11] Arguments: [cs_warehouse_sk#9, cs_order_number#10], [cs_warehouse_sk#9, cs_order_number#10] -(9) ColumnarToRow [codegen id : 3] +(9) CometColumnarExchange Input [2]: [cs_warehouse_sk#9, cs_order_number#10] +Arguments: hashpartitioning(cs_order_number#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(10) Exchange +(10) CometSort Input [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: hashpartitioning(cs_order_number#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: [cs_warehouse_sk#9, cs_order_number#10], [cs_order_number#10 ASC NULLS FIRST] -(11) Sort [codegen id : 4] +(11) ColumnarToRow [codegen id : 2] Input [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: [cs_order_number#10 ASC NULLS FIRST], false, 0 -(12) SortMergeJoin [codegen id : 5] +(12) SortMergeJoin [codegen id : 3] Left keys [1]: [cs_order_number#5] Right keys [1]: [cs_order_number#10] Join type: LeftSemi Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#9) -(13) Project [codegen id : 5] +(13) Project [codegen id : 3] Output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] @@ -113,18 +115,18 @@ ReadSchema: struct Input [2]: [cr_order_number#12, cr_returned_date_sk#13] Arguments: [cr_order_number#12], [cr_order_number#12] -(16) ColumnarToRow [codegen id : 6] +(16) CometColumnarExchange Input [1]: [cr_order_number#12] +Arguments: hashpartitioning(cr_order_number#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(17) Exchange +(17) CometSort Input [1]: [cr_order_number#12] -Arguments: hashpartitioning(cr_order_number#12, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: [cr_order_number#12], [cr_order_number#12 ASC NULLS FIRST] -(18) Sort [codegen id : 7] +(18) ColumnarToRow [codegen id : 4] Input [1]: [cr_order_number#12] -Arguments: [cr_order_number#12 ASC NULLS FIRST], false, 0 -(19) SortMergeJoin [codegen id : 11] +(19) SortMergeJoin [codegen id : 8] Left keys [1]: [cs_order_number#5] Right keys [1]: [cr_order_number#12] Join type: LeftAnti @@ -145,20 +147,20 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2002-02-01)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(23) ColumnarToRow [codegen id : 8] +(23) ColumnarToRow [codegen id : 5] Input [1]: [d_date_sk#14] (24) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(25) BroadcastHashJoin [codegen id : 11] +(25) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_ship_date_sk#1] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(26) Project [codegen id : 11] +(26) Project [codegen id : 8] Output [5]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#14] @@ -177,20 +179,20 @@ Condition : ((isnotnull(ca_state#17) AND (ca_state#17 = GA)) AND isnotnull(ca_ad Input [2]: [ca_address_sk#16, ca_state#17] Arguments: [ca_address_sk#16], [ca_address_sk#16] -(30) ColumnarToRow [codegen id : 9] +(30) ColumnarToRow [codegen id : 6] Input [1]: [ca_address_sk#16] (31) BroadcastExchange Input [1]: [ca_address_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(32) BroadcastHashJoin [codegen id : 11] +(32) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_ship_addr_sk#2] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(33) Project [codegen id : 11] +(33) Project [codegen id : 8] Output [4]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [6]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#16] @@ -209,49 +211,55 @@ Condition : ((isnotnull(cc_county#19) AND (cc_county#19 = Williamson County)) AN Input [2]: [cc_call_center_sk#18, cc_county#19] Arguments: [cc_call_center_sk#18], [cc_call_center_sk#18] -(37) ColumnarToRow [codegen id : 10] +(37) ColumnarToRow [codegen id : 7] Input [1]: [cc_call_center_sk#18] (38) BroadcastExchange Input [1]: [cc_call_center_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(39) BroadcastHashJoin [codegen id : 11] +(39) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_call_center_sk#3] Right keys [1]: [cc_call_center_sk#18] Join type: Inner Join condition: None -(40) Project [codegen id : 11] +(40) Project [codegen id : 8] Output [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [5]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#18] -(41) HashAggregate [codegen id : 11] +(41) HashAggregate [codegen id : 8] Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Keys [1]: [cs_order_number#5] Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#20, sum(UnscaledValue(cs_net_profit#7))#21] Results [3]: [cs_order_number#5, sum#22, sum#23] -(42) HashAggregate [codegen id : 11] +(42) HashAggregate [codegen id : 8] Input [3]: [cs_order_number#5, sum#22, sum#23] Keys [1]: [cs_order_number#5] Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#20, sum(UnscaledValue(cs_net_profit#7))#21] Results [3]: [cs_order_number#5, sum#22, sum#23] -(43) HashAggregate [codegen id : 11] +(43) HashAggregate [codegen id : 8] Input [3]: [cs_order_number#5, sum#22, sum#23] Keys: [] Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#20, sum(UnscaledValue(cs_net_profit#7))#21, count(cs_order_number#5)#24] Results [3]: [sum#22, sum#23, count#25] -(44) Exchange +(44) RowToColumnar Input [3]: [sum#22, sum#23, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(45) HashAggregate [codegen id : 12] +(45) CometColumnarExchange +Input [3]: [sum#22, sum#23, count#25] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(46) ColumnarToRow [codegen id : 9] +Input [3]: [sum#22, sum#23, count#25] + +(47) HashAggregate [codegen id : 9] Input [3]: [sum#22, sum#23, count#25] Keys: [] Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/simplified.txt index 8935abb54..605ce3703 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/simplified.txt @@ -1,74 +1,70 @@ -WholeStageCodegen (12) +WholeStageCodegen (9) HashAggregate [sum,sum,count] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] - InputAdapter - Exchange #1 - WholeStageCodegen (11) - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] - HashAggregate [cs_order_number,cs_ext_ship_cost,cs_net_profit] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] - Project [cs_order_number,cs_ext_ship_cost,cs_net_profit] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - BroadcastHashJoin [cs_ship_addr_sk,ca_address_sk] - Project [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk] - SortMergeJoin [cs_order_number,cr_order_number] - InputAdapter - WholeStageCodegen (5) - Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - SortMergeJoin [cs_order_number,cs_order_number,cs_warehouse_sk,cs_warehouse_sk] - InputAdapter - WholeStageCodegen (2) - Sort [cs_order_number] - InputAdapter - Exchange [cs_order_number] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometColumnarExchange #1 + RowToColumnar + WholeStageCodegen (8) + HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] + HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] + HashAggregate [cs_order_number,cs_ext_ship_cost,cs_net_profit] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] + Project [cs_order_number,cs_ext_ship_cost,cs_net_profit] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + BroadcastHashJoin [cs_ship_addr_sk,ca_address_sk] + Project [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk] + SortMergeJoin [cs_order_number,cr_order_number] + InputAdapter + WholeStageCodegen (3) + Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + SortMergeJoin [cs_order_number,cs_order_number,cs_warehouse_sk,cs_warehouse_sk] + InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometSort [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + CometColumnarExchange [cs_order_number] #2 CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] - InputAdapter - WholeStageCodegen (4) - Sort [cs_order_number] - InputAdapter - Exchange [cs_order_number] #3 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter + InputAdapter + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometSort [cs_warehouse_sk,cs_order_number] + CometColumnarExchange [cs_order_number] #3 CometProject [cs_warehouse_sk,cs_order_number] CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] - InputAdapter - WholeStageCodegen (7) - Sort [cr_order_number] - InputAdapter - Exchange [cr_order_number] #4 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter + InputAdapter + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometSort [cr_order_number] + CometColumnarExchange [cr_order_number] #4 CometProject [cr_order_number] CometScan parquet spark_catalog.default.catalog_returns [cr_order_number,cr_returned_date_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (8) + BroadcastExchange #6 + WholeStageCodegen (6) ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) + BroadcastExchange #7 + WholeStageCodegen (7) ColumnarToRow InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometProject [cc_call_center_sk] - CometFilter [cc_call_center_sk,cc_county] - CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_county] + CometProject [cc_call_center_sk] + CometFilter [cc_call_center_sk,cc_county] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_county] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/explain.txt index 8d9edc0a0..ddf62fe94 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * HashAggregate (41) - +- Exchange (40) - +- * ColumnarToRow (39) +* ColumnarToRow (42) ++- CometTakeOrderedAndProject (41) + +- CometHashAggregate (40) + +- CometColumnarExchange (39) +- CometHashAggregate (38) +- CometProject (37) +- CometBroadcastHashJoin (36) @@ -226,23 +226,21 @@ Input [6]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25, i_ 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))] -(39) ColumnarToRow [codegen id : 1] +(39) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(41) HashAggregate [codegen id : 2] +(40) CometHashAggregate 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)#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] -(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] +(41) CometTakeOrderedAndProject +Input [15]: [i_item_id#27, i_item_desc#28, s_state#25, store_sales_quantitycount#47, store_sales_quantityave#48, store_sales_quantitystdev#49, store_sales_quantitycov#50, as_store_returns_quantitycount#51, as_store_returns_quantityave#52, as_store_returns_quantitystdev#53, store_returns_quantitycov#54, catalog_sales_quantitycount#55, catalog_sales_quantityave#56, catalog_sales_quantitystdev#57, catalog_sales_quantitycov#58] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#27 ASC NULLS FIRST,i_item_desc#28 ASC NULLS FIRST,s_state#25 ASC NULLS FIRST], output=[i_item_id#27,i_item_desc#28,s_state#25,store_sales_quantitycount#47,store_sales_quantityave#48,store_sales_quantitystdev#49,store_sales_quantitycov#50,as_store_returns_quantitycount#51,as_store_returns_quantityave#52,as_store_returns_quantitystdev#53,store_returns_quantitycov#54,catalog_sales_quantitycount#55,catalog_sales_quantityave#56,catalog_sales_quantitystdev#57,catalog_sales_quantitycov#58]), [i_item_id#27, i_item_desc#28, s_state#25, store_sales_quantitycount#47, store_sales_quantityave#48, store_sales_quantitystdev#49, store_sales_quantitycov#50, as_store_returns_quantitycount#51, as_store_returns_quantityave#52, as_store_returns_quantitystdev#53, store_returns_quantitycov#54, catalog_sales_quantitycount#55, catalog_sales_quantityave#56, catalog_sales_quantitystdev#57, catalog_sales_quantitycov#58], 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#47, store_sales_quantityave#48, store_sales_quantitystdev#49, store_sales_quantitycov#50, as_store_returns_quantitycount#51, as_store_returns_quantityave#52, as_store_returns_quantitystdev#53, store_returns_quantitycov#54, catalog_sales_quantitycount#55, catalog_sales_quantityave#56, catalog_sales_quantitystdev#57, catalog_sales_quantitycov#58] + +(42) ColumnarToRow [codegen id : 1] +Input [15]: [i_item_id#27, i_item_desc#28, s_state#25, store_sales_quantitycount#47, store_sales_quantityave#48, store_sales_quantitystdev#49, store_sales_quantitycov#50, as_store_returns_quantitycount#51, as_store_returns_quantityave#52, as_store_returns_quantitystdev#53, store_returns_quantitycov#54, catalog_sales_quantitycount#55, catalog_sales_quantityave#56, catalog_sales_quantitystdev#57, catalog_sales_quantitycov#58] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/simplified.txt index ae6bab279..e350dfe51 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/simplified.txt @@ -1,63 +1,61 @@ -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 (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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [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,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,ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_sk,i_item_id,i_item_desc] - CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_state] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,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 [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_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_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_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 [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,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 - CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - 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_date_sk,d_quarter_name] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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 [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange [d_date_sk] #7 +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [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] + CometHashAggregate [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,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))] + CometColumnarExchange [i_item_id,i_item_desc,s_state] #1 + CometHashAggregate [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,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,ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_sk,i_item_id,i_item_desc] + CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_state] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,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 [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_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_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_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 [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,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_quarter_name] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + 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_date_sk,d_quarter_name] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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 [d_date_sk] #6 CometProject [d_date_sk] CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [s_store_sk,s_state] #8 - CometFilter [s_store_sk,s_state] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] + CometBroadcastExchange [d_date_sk] #7 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_quarter_name] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange [s_store_sk,s_state] #8 + CometFilter [s_store_sk,s_state] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 + CometFilter [i_item_sk,i_item_id,i_item_desc] + 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-spark3_5/q18/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/explain.txt index c09ef1445..6c969939d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (41) -+- * HashAggregate (40) - +- Exchange (39) - +- * ColumnarToRow (38) +* ColumnarToRow (41) ++- CometTakeOrderedAndProject (40) + +- CometHashAggregate (39) + +- CometColumnarExchange (38) +- CometHashAggregate (37) +- CometExpand (36) +- CometProject (35) @@ -219,23 +219,21 @@ Input [12]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, 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)))] -(38) ColumnarToRow [codegen id : 1] +(38) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(39) CometHashAggregate 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)))#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] -(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] +(40) CometTakeOrderedAndProject +Input [11]: [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, agg1#48, agg2#49, agg3#50, agg4#51, agg5#52, agg6#53, agg7#54] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[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], output=[i_item_id#29,ca_country#30,ca_state#31,ca_county#32,agg1#48,agg2#49,agg3#50,agg4#51,agg5#52,agg6#53,agg7#54]), [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, agg1#48, agg2#49, agg3#50, agg4#51, agg5#52, agg6#53, agg7#54], 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#48, agg2#49, agg3#50, agg4#51, agg5#52, agg6#53, agg7#54] + +(41) ColumnarToRow [codegen id : 1] +Input [11]: [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, agg1#48, agg2#49, agg3#50, agg4#51, agg5#52, agg6#53, agg7#54] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/simplified.txt index 2519d58a2..8182da775 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/simplified.txt @@ -1,53 +1,51 @@ -TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [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,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,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,i_item_sk,i_item_id] - 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_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,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 [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,ca_address_sk,ca_county,ca_state,ca_country] - 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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - 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_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,cd_demo_sk,cd_dep_count] - CometFilter [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] - 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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange [cd_demo_sk] #5 - CometFilter [cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #8 - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,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)))] + CometColumnarExchange [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] #1 + CometHashAggregate [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,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,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,i_item_sk,i_item_id] + 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_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,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 [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,ca_address_sk,ca_county,ca_state,ca_country] + 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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + 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_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,cd_demo_sk,cd_dep_count] + CometFilter [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] + 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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 + CometProject [cd_demo_sk,cd_dep_count] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometBroadcastExchange [cd_demo_sk] #5 + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] + CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] + CometBroadcastExchange [d_date_sk] #7 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_item_id] #8 + CometFilter [i_item_sk,i_item_id] + 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-spark3_5/q19/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/explain.txt index a00474bee..2b3480d80 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (34) -+- * HashAggregate (33) - +- Exchange (32) - +- * ColumnarToRow (31) +* ColumnarToRow (34) ++- CometTakeOrderedAndProject (33) + +- CometHashAggregate (32) + +- CometColumnarExchange (31) +- CometHashAggregate (30) +- CometProject (29) +- CometBroadcastHashJoin (28) @@ -180,21 +180,19 @@ Input [5]: [ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i 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))] -(31) ColumnarToRow [codegen id : 1] +(31) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(33) HashAggregate [codegen id : 2] +(32) CometHashAggregate 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))#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] -(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] +(33) CometTakeOrderedAndProject +Input [5]: [brand_id#22, brand#23, i_manufact_id#12, i_manufact#13, ext_price#24] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#24 DESC NULLS LAST,brand#23 ASC NULLS FIRST,brand_id#22 ASC NULLS FIRST,i_manufact_id#12 ASC NULLS FIRST,i_manufact#13 ASC NULLS FIRST], output=[brand_id#22,brand#23,i_manufact_id#12,i_manufact#13,ext_price#24]), [brand_id#22, brand#23, i_manufact_id#12, i_manufact#13, ext_price#24], 100, [ext_price#24 DESC NULLS LAST, brand#23 ASC NULLS FIRST, brand_id#22 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST, i_manufact#13 ASC NULLS FIRST], [brand_id#22, brand#23, i_manufact_id#12, i_manufact#13, ext_price#24] + +(34) ColumnarToRow [codegen id : 1] +Input [5]: [brand_id#22, brand#23, i_manufact_id#12, i_manufact#13, ext_price#24] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/simplified.txt index 163a31f47..8f81e32ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/simplified.txt @@ -1,38 +1,36 @@ -TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,sum,ss_ext_sales_price] - CometProject [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip,s_store_sk,s_zip] - CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk,ca_address_sk,ca_zip] - 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,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_customer_sk,c_current_addr_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,ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_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 [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] #3 - CometProject [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #4 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_zip] #5 - CometFilter [ca_address_sk,ca_zip] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange [s_store_sk,s_zip] #6 - CometFilter [s_store_sk,s_zip] - CometScan parquet spark_catalog.default.store [s_store_sk,s_zip] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [brand_id,brand,i_manufact_id,i_manufact,ext_price] + CometHashAggregate [brand_id,brand,i_manufact_id,i_manufact,ext_price,i_brand,i_brand_id,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange [i_brand,i_brand_id,i_manufact_id,i_manufact] #1 + CometHashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,sum,ss_ext_sales_price] + CometProject [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip,s_store_sk,s_zip] + CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk,ca_address_sk,ca_zip] + 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,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_customer_sk,c_current_addr_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,ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] + CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_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 [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] #3 + CometProject [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #4 + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange [ca_address_sk,ca_zip] #5 + CometFilter [ca_address_sk,ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] + CometBroadcastExchange [s_store_sk,s_zip] #6 + CometFilter [s_store_sk,s_zip] + 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-spark3_5/q2/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/explain.txt index 8565313c9..d37261260 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/explain.txt @@ -1,39 +1,37 @@ == Physical Plan == -* 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) +* ColumnarToRow (33) ++- CometSort (32) + +- CometColumnarExchange (31) + +- CometProject (30) + +- CometBroadcastHashJoin (29) + :- CometProject (19) + : +- CometBroadcastHashJoin (18) + : :- CometHashAggregate (13) + : : +- CometColumnarExchange (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) + : +- CometBroadcastExchange (17) + : +- CometProject (16) + : +- CometFilter (15) + : +- CometScan parquet spark_catalog.default.date_dim (14) + +- CometBroadcastExchange (28) + +- CometProject (27) + +- CometBroadcastHashJoin (26) + :- CometHashAggregate (21) + : +- ReusedExchange (20) + +- CometBroadcastExchange (25) + +- CometProject (24) + +- CometFilter (23) + +- CometScan parquet spark_catalog.default.date_dim (22) (1) Scan parquet spark_catalog.default.web_sales @@ -91,113 +89,100 @@ 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))] -(12) ColumnarToRow [codegen id : 1] +(12) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(14) HashAggregate [codegen id : 6] +(13) CometHashAggregate 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))#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] -(15) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#33, d_year#34] +(14) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_week_seq#19, d_year#20] 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 -(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)) - -(17) CometProject -Input [2]: [d_week_seq#33, d_year#34] -Arguments: [d_week_seq#33], [d_week_seq#33] +(15) CometFilter +Input [2]: [d_week_seq#19, d_year#20] +Condition : ((isnotnull(d_year#20) AND (d_year#20 = 2001)) AND isnotnull(d_week_seq#19)) -(18) ColumnarToRow [codegen id : 2] -Input [1]: [d_week_seq#33] +(16) CometProject +Input [2]: [d_week_seq#19, d_year#20] +Arguments: [d_week_seq#19], [d_week_seq#19] -(19) BroadcastExchange -Input [1]: [d_week_seq#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(17) CometBroadcastExchange +Input [1]: [d_week_seq#19] +Arguments: [d_week_seq#19] -(20) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [d_week_seq#10] -Right keys [1]: [d_week_seq#33] -Join type: Inner -Join condition: None +(18) CometBroadcastHashJoin +Left output [8]: [d_week_seq#10, sun_sales#21, mon_sales#22, tue_sales#23, wed_sales#24, thu_sales#25, fri_sales#26, sat_sales#27] +Right output [1]: [d_week_seq#19] +Arguments: [d_week_seq#10], [d_week_seq#19], Inner, BuildRight -(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] +(19) CometProject +Input [9]: [d_week_seq#10, sun_sales#21, mon_sales#22, tue_sales#23, wed_sales#24, thu_sales#25, fri_sales#26, sat_sales#27, d_week_seq#19] +Arguments: [d_week_seq1#28, sun_sales1#29, mon_sales1#30, tue_sales1#31, wed_sales1#32, thu_sales1#33, fri_sales1#34, sat_sales1#35], [d_week_seq#10 AS d_week_seq1#28, sun_sales#21 AS sun_sales1#29, mon_sales#22 AS mon_sales1#30, tue_sales#23 AS tue_sales1#31, wed_sales#24 AS wed_sales1#32, thu_sales#25 AS thu_sales1#33, fri_sales#26 AS fri_sales1#34, sat_sales#27 AS sat_sales1#35] -(22) ReusedExchange [Reuses operator id: 13] -Output [8]: [d_week_seq#43, sum#44, sum#45, sum#46, sum#47, sum#48, sum#49, sum#50] +(20) ReusedExchange [Reuses operator id: 12] +Output [8]: [d_week_seq#36, sum#37, sum#38, sum#39, sum#40, sum#41, sum#42, sum#43] -(23) HashAggregate [codegen id : 5] -Input [8]: [d_week_seq#43, sum#44, sum#45, sum#46, sum#47, sum#48, sum#49, sum#50] -Keys [1]: [d_week_seq#43] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#51 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#51 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#51 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#51 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#51 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#51 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#51 = Saturday ) THEN sales_price#4 END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#51 = Sunday ) THEN sales_price#4 END))#19, sum(UnscaledValue(CASE WHEN (d_day_name#51 = Monday ) THEN sales_price#4 END))#20, sum(UnscaledValue(CASE WHEN (d_day_name#51 = Tuesday ) THEN sales_price#4 END))#21, sum(UnscaledValue(CASE WHEN (d_day_name#51 = Wednesday) THEN sales_price#4 END))#22, sum(UnscaledValue(CASE WHEN (d_day_name#51 = Thursday ) THEN sales_price#4 END))#23, sum(UnscaledValue(CASE WHEN (d_day_name#51 = Friday ) THEN sales_price#4 END))#24, sum(UnscaledValue(CASE WHEN (d_day_name#51 = Saturday ) THEN sales_price#4 END))#25] -Results [8]: [d_week_seq#43, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#51 = Sunday ) THEN sales_price#4 END))#19,17,2) AS sun_sales#52, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#51 = Monday ) THEN sales_price#4 END))#20,17,2) AS mon_sales#53, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#51 = Tuesday ) THEN sales_price#4 END))#21,17,2) AS tue_sales#54, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#51 = Wednesday) THEN sales_price#4 END))#22,17,2) AS wed_sales#55, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#51 = Thursday ) THEN sales_price#4 END))#23,17,2) AS thu_sales#56, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#51 = Friday ) THEN sales_price#4 END))#24,17,2) AS fri_sales#57, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#51 = Saturday ) THEN sales_price#4 END))#25,17,2) AS sat_sales#58] +(21) CometHashAggregate +Input [8]: [d_week_seq#36, sum#37, sum#38, sum#39, sum#40, sum#41, sum#42, sum#43] +Keys [1]: [d_week_seq#36] +Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#44 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#44 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#44 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#44 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#44 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#44 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#44 = Saturday ) THEN sales_price#4 END))] -(24) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#59, d_year#60] +(22) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_week_seq#45, d_year#46] 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 -(25) CometFilter -Input [2]: [d_week_seq#59, d_year#60] -Condition : ((isnotnull(d_year#60) AND (d_year#60 = 2002)) AND isnotnull(d_week_seq#59)) +(23) CometFilter +Input [2]: [d_week_seq#45, d_year#46] +Condition : ((isnotnull(d_year#46) AND (d_year#46 = 2002)) AND isnotnull(d_week_seq#45)) -(26) CometProject -Input [2]: [d_week_seq#59, d_year#60] -Arguments: [d_week_seq#59], [d_week_seq#59] +(24) CometProject +Input [2]: [d_week_seq#45, d_year#46] +Arguments: [d_week_seq#45], [d_week_seq#45] -(27) ColumnarToRow [codegen id : 4] -Input [1]: [d_week_seq#59] +(25) CometBroadcastExchange +Input [1]: [d_week_seq#45] +Arguments: [d_week_seq#45] -(28) BroadcastExchange -Input [1]: [d_week_seq#59] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(26) CometBroadcastHashJoin +Left output [8]: [d_week_seq#36, sun_sales#47, mon_sales#48, tue_sales#49, wed_sales#50, thu_sales#51, fri_sales#52, sat_sales#53] +Right output [1]: [d_week_seq#45] +Arguments: [d_week_seq#36], [d_week_seq#45], Inner, BuildRight -(29) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [d_week_seq#43] -Right keys [1]: [d_week_seq#59] -Join type: Inner -Join condition: None +(27) CometProject +Input [9]: [d_week_seq#36, sun_sales#47, mon_sales#48, tue_sales#49, wed_sales#50, thu_sales#51, fri_sales#52, sat_sales#53, d_week_seq#45] +Arguments: [d_week_seq2#54, sun_sales2#55, mon_sales2#56, tue_sales2#57, wed_sales2#58, thu_sales2#59, fri_sales2#60, sat_sales2#61], [d_week_seq#36 AS d_week_seq2#54, sun_sales#47 AS sun_sales2#55, mon_sales#48 AS mon_sales2#56, tue_sales#49 AS tue_sales2#57, wed_sales#50 AS wed_sales2#58, thu_sales#51 AS thu_sales2#59, fri_sales#52 AS fri_sales2#60, sat_sales#53 AS sat_sales2#61] -(30) Project [codegen id : 5] -Output [8]: [d_week_seq#43 AS d_week_seq2#61, sun_sales#52 AS sun_sales2#62, mon_sales#53 AS mon_sales2#63, tue_sales#54 AS tue_sales2#64, wed_sales#55 AS wed_sales2#65, thu_sales#56 AS thu_sales2#66, fri_sales#57 AS fri_sales2#67, sat_sales#58 AS sat_sales2#68] -Input [9]: [d_week_seq#43, sun_sales#52, mon_sales#53, tue_sales#54, wed_sales#55, thu_sales#56, fri_sales#57, sat_sales#58, d_week_seq#59] +(28) CometBroadcastExchange +Input [8]: [d_week_seq2#54, sun_sales2#55, mon_sales2#56, tue_sales2#57, wed_sales2#58, thu_sales2#59, fri_sales2#60, sat_sales2#61] +Arguments: [d_week_seq2#54, sun_sales2#55, mon_sales2#56, tue_sales2#57, wed_sales2#58, thu_sales2#59, fri_sales2#60, sat_sales2#61] -(31) BroadcastExchange -Input [8]: [d_week_seq2#61, sun_sales2#62, mon_sales2#63, tue_sales2#64, wed_sales2#65, thu_sales2#66, fri_sales2#67, sat_sales2#68] -Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [plan_id=4] +(29) CometBroadcastHashJoin +Left output [8]: [d_week_seq1#28, sun_sales1#29, mon_sales1#30, tue_sales1#31, wed_sales1#32, thu_sales1#33, fri_sales1#34, sat_sales1#35] +Right output [8]: [d_week_seq2#54, sun_sales2#55, mon_sales2#56, tue_sales2#57, wed_sales2#58, thu_sales2#59, fri_sales2#60, sat_sales2#61] +Arguments: [d_week_seq1#28], [(d_week_seq2#54 - 53)], Inner, BuildRight -(32) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [d_week_seq1#35] -Right keys [1]: [(d_week_seq2#61 - 53)] -Join type: Inner -Join condition: None +(30) CometProject +Input [16]: [d_week_seq1#28, sun_sales1#29, mon_sales1#30, tue_sales1#31, wed_sales1#32, thu_sales1#33, fri_sales1#34, sat_sales1#35, d_week_seq2#54, sun_sales2#55, mon_sales2#56, tue_sales2#57, wed_sales2#58, thu_sales2#59, fri_sales2#60, sat_sales2#61] +Arguments: [d_week_seq1#28, round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1 / sat_sales2), 2)#68], [d_week_seq1#28, round((sun_sales1#29 / sun_sales2#55), 2) AS round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1#30 / mon_sales2#56), 2) AS round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1#31 / tue_sales2#57), 2) AS round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1#32 / wed_sales2#58), 2) AS round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1#33 / thu_sales2#59), 2) AS round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1#34 / fri_sales2#60), 2) AS round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1#35 / sat_sales2#61), 2) AS round((sat_sales1 / sat_sales2), 2)#68] -(33) Project [codegen id : 6] -Output [8]: [d_week_seq1#35, round((sun_sales1#36 / sun_sales2#62), 2) AS round((sun_sales1 / sun_sales2), 2)#69, round((mon_sales1#37 / mon_sales2#63), 2) AS round((mon_sales1 / mon_sales2), 2)#70, round((tue_sales1#38 / tue_sales2#64), 2) AS round((tue_sales1 / tue_sales2), 2)#71, round((wed_sales1#39 / wed_sales2#65), 2) AS round((wed_sales1 / wed_sales2), 2)#72, round((thu_sales1#40 / thu_sales2#66), 2) AS round((thu_sales1 / thu_sales2), 2)#73, round((fri_sales1#41 / fri_sales2#67), 2) AS round((fri_sales1 / fri_sales2), 2)#74, round((sat_sales1#42 / sat_sales2#68), 2) AS round((sat_sales1 / sat_sales2), 2)#75] -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#61, sun_sales2#62, mon_sales2#63, tue_sales2#64, wed_sales2#65, thu_sales2#66, fri_sales2#67, sat_sales2#68] +(31) CometColumnarExchange +Input [8]: [d_week_seq1#28, round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1 / sat_sales2), 2)#68] +Arguments: rangepartitioning(d_week_seq1#28 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(34) Exchange -Input [8]: [d_week_seq1#35, round((sun_sales1 / sun_sales2), 2)#69, round((mon_sales1 / mon_sales2), 2)#70, round((tue_sales1 / tue_sales2), 2)#71, round((wed_sales1 / wed_sales2), 2)#72, round((thu_sales1 / thu_sales2), 2)#73, round((fri_sales1 / fri_sales2), 2)#74, round((sat_sales1 / sat_sales2), 2)#75] -Arguments: rangepartitioning(d_week_seq1#35 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(32) CometSort +Input [8]: [d_week_seq1#28, round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1 / sat_sales2), 2)#68] +Arguments: [d_week_seq1#28, round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1 / sat_sales2), 2)#68], [d_week_seq1#28 ASC NULLS FIRST] -(35) Sort [codegen id : 7] -Input [8]: [d_week_seq1#35, round((sun_sales1 / sun_sales2), 2)#69, round((mon_sales1 / mon_sales2), 2)#70, round((tue_sales1 / tue_sales2), 2)#71, round((wed_sales1 / wed_sales2), 2)#72, round((thu_sales1 / thu_sales2), 2)#73, round((fri_sales1 / fri_sales2), 2)#74, round((sat_sales1 / sat_sales2), 2)#75] -Arguments: [d_week_seq1#35 ASC NULLS FIRST], true, 0 +(33) ColumnarToRow [codegen id : 1] +Input [8]: [d_week_seq1#28, round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1 / sat_sales2), 2)#68] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/simplified.txt index 5d0658192..c5c06a0b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/simplified.txt @@ -1,50 +1,35 @@ -WholeStageCodegen (7) - Sort [d_week_seq1] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [d_week_seq1] #1 - 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] - 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] - InputAdapter - Exchange [d_week_seq] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum,d_day_name,sales_price] - CometProject [sales_price,d_week_seq,d_day_name] - CometBroadcastHashJoin [sold_date_sk,sales_price,d_date_sk,d_week_seq,d_day_name] - CometUnion [sold_date_sk,sales_price] - 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] - CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 - CometFilter [d_date_sk,d_week_seq,d_day_name] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year] - CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] - InputAdapter - BroadcastExchange #5 - 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] - InputAdapter - ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year] - CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] + CometSort [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] + CometColumnarExchange [d_week_seq1] #1 + CometProject [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] [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] + CometBroadcastHashJoin [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] + CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] + CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] + CometHashAggregate [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,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))] + CometColumnarExchange [d_week_seq] #2 + CometHashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum,d_day_name,sales_price] + CometProject [sales_price,d_week_seq,d_day_name] + CometBroadcastHashJoin [sold_date_sk,sales_price,d_date_sk,d_week_seq,d_day_name] + CometUnion [sold_date_sk,sales_price] + 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] + CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 + CometFilter [d_date_sk,d_week_seq,d_day_name] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] + CometBroadcastExchange [d_week_seq] #4 + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] + CometBroadcastExchange [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 + CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] + CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] + CometHashAggregate [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,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))] + ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 + CometBroadcastExchange [d_week_seq] #6 + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/explain.txt index 7c29ff218..c6982f073 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/explain.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject (22) +- * Project (21) +- Window (20) - +- * Sort (19) - +- Exchange (18) - +- * HashAggregate (17) - +- Exchange (16) - +- * ColumnarToRow (15) + +- * ColumnarToRow (19) + +- CometSort (18) + +- CometColumnarExchange (17) + +- CometHashAggregate (16) + +- CometColumnarExchange (15) +- CometHashAggregate (14) +- CometProject (13) +- CometBroadcastHashJoin (12) @@ -92,39 +92,37 @@ Input [6]: [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, 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))] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(16) CometHashAggregate 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] -(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=2] +(17) CometColumnarExchange +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(18) CometSort +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] +Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6], [i_class#9 ASC NULLS FIRST] -(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 +(19) ColumnarToRow [codegen id : 1] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] (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] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] +Arguments: [sum(_w0#15) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#16], [i_class#9] -(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] +(21) Project [codegen id : 2] +Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, ((_w0#15 * 100) / _we0#16) AS revenueratio#17, i_item_id#6] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6, _we0#16] (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] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17, 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#17 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/simplified.txt index 08088a386..9c30689f6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/simplified.txt @@ -1,38 +1,34 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (4) + WholeStageCodegen (2) Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] InputAdapter Window [_w0,i_class] - WholeStageCodegen (3) - Sort [i_class] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [i_class] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,cs_ext_sales_price] - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometColumnarExchange [i_class] #1 + CometHashAggregate [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum,sum(UnscaledValue(cs_ext_sales_price))] + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,cs_ext_sales_price] + CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + 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-spark3_5/q21/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/explain.txt index 21c979264..ff0f63b74 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * Filter (23) - +- * HashAggregate (22) - +- Exchange (21) - +- * ColumnarToRow (20) +* ColumnarToRow (24) ++- CometTakeOrderedAndProject (23) + +- CometFilter (22) + +- CometHashAggregate (21) + +- CometColumnarExchange (20) +- CometHashAggregate (19) +- CometProject (18) +- CometBroadcastHashJoin (17) @@ -118,27 +118,25 @@ 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)] -(20) ColumnarToRow [codegen id : 1] +(20) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(21) Exchange -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 : 2] +(21) CometHashAggregate 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)#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 : 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) +(22) CometFilter +Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#15, inv_after#16] +Condition : (CASE WHEN (inv_before#15 > 0) THEN ((cast(inv_after#16 as double) / cast(inv_before#15 as double)) >= 0.666667) END AND CASE WHEN (inv_before#15 > 0) THEN ((cast(inv_after#16 as double) / cast(inv_before#15 as double)) <= 1.5) END) + +(23) CometTakeOrderedAndProject +Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#15, inv_after#16] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_warehouse_name#7 ASC NULLS FIRST,i_item_id#9 ASC NULLS FIRST], output=[w_warehouse_name#7,i_item_id#9,inv_before#15,inv_after#16]), [w_warehouse_name#7, i_item_id#9, inv_before#15, inv_after#16], 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#15, inv_after#16] -(24) TakeOrderedAndProject -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] +(24) ColumnarToRow [codegen id : 1] +Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#15, inv_after#16] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/simplified.txt index bd81a3a7b..1a4f175c4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/simplified.txt @@ -1,35 +1,33 @@ -TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [w_warehouse_name,i_item_id,sum,sum,d_date,inv_quantity_on_hand] - CometProject [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] - CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id,d_date_sk,d_date] - CometProject [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id] - CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_sk,i_item_id] - CometProject [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_sk,w_warehouse_name] - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #3 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [i_item_sk,i_item_id] #4 - CometProject [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id,i_current_price] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - CometBroadcastExchange [d_date_sk,d_date] #5 - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] + CometFilter [w_warehouse_name,i_item_id,inv_before,inv_after] + CometHashAggregate [w_warehouse_name,i_item_id,inv_before,inv_after,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)] + CometColumnarExchange [w_warehouse_name,i_item_id] #1 + CometHashAggregate [w_warehouse_name,i_item_id,sum,sum,d_date,inv_quantity_on_hand] + CometProject [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id,d_date_sk,d_date] + CometProject [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_sk,i_item_id] + CometProject [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #3 + CometFilter [w_warehouse_sk,w_warehouse_name] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange [i_item_sk,i_item_id] #4 + CometProject [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id,i_current_price] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] + CometBroadcastExchange [d_date_sk,d_date] #5 + CometFilter [d_date_sk,d_date] + 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-spark3_5/q22/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/explain.txt index df6a80179..1f7209c62 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * HashAggregate (23) - +- Exchange (22) - +- * ColumnarToRow (21) +* ColumnarToRow (24) ++- CometTakeOrderedAndProject (23) + +- CometHashAggregate (22) + +- CometColumnarExchange (21) +- CometHashAggregate (20) +- CometExpand (19) +- CometProject (18) @@ -122,23 +122,21 @@ Input [6]: [inv_quantity_on_hand#3, i_product_name#14, i_brand#15, i_class#16, i 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)] -(21) ColumnarToRow [codegen id : 1] +(21) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(23) HashAggregate [codegen id : 2] +(22) CometHashAggregate 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)#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] -(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] +(23) CometTakeOrderedAndProject +Input [5]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, qoh#21] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#21 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], output=[i_product_name#14,i_brand#15,i_class#16,i_category#17,qoh#21]), [i_product_name#14, i_brand#15, i_class#16, i_category#17, qoh#21], 100, [qoh#21 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#21] + +(24) ColumnarToRow [codegen id : 1] +Input [5]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, qoh#21] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/simplified.txt index 7d36dc400..05d02283b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/simplified.txt @@ -1,36 +1,34 @@ -TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count,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,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] - CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] - CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange [w_warehouse_sk] #5 - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_product_name,i_brand,i_class,i_category,qoh] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh,spark_grouping_id,sum,count,avg(inv_quantity_on_hand)] + CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 + CometHashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count,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,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] + CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] + CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometBroadcastExchange [w_warehouse_sk] #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-spark3_5/q23a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/explain.txt index 9c7a61130..2af9c6082 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/explain.txt @@ -1,71 +1,71 @@ == Physical Plan == -* 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 (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) +* ColumnarToRow (67) ++- CometHashAggregate (66) + +- CometColumnarExchange (65) + +- CometHashAggregate (64) + +- CometUnion (63) + :- CometProject (46) + : +- CometBroadcastHashJoin (45) + : :- CometProject (40) + : : +- CometSortMergeJoin (39) + : : :- CometSort (24) + : : : +- CometColumnarExchange (23) + : : : +- CometProject (22) + : : : +- CometBroadcastHashJoin (21) + : : : :- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : +- CometBroadcastExchange (20) + : : : +- CometProject (19) + : : : +- CometFilter (18) + : : : +- CometHashAggregate (17) + : : : +- CometColumnarExchange (16) + : : : +- CometHashAggregate (15) + : : : +- CometProject (14) + : : : +- CometBroadcastHashJoin (13) + : : : :- CometProject (9) + : : : : +- CometBroadcastHashJoin (8) + : : : : :- CometFilter (3) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (2) + : : : : +- CometBroadcastExchange (7) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.date_dim (4) + : : : +- CometBroadcastExchange (12) + : : : +- CometFilter (11) + : : : +- CometScan parquet spark_catalog.default.item (10) + : : +- CometSort (38) + : : +- CometProject (37) + : : +- CometFilter (36) + : : +- CometHashAggregate (35) + : : +- CometColumnarExchange (34) + : : +- CometHashAggregate (33) + : : +- CometProject (32) + : : +- CometBroadcastHashJoin (31) + : : :- CometProject (27) + : : : +- CometFilter (26) + : : : +- CometScan parquet spark_catalog.default.store_sales (25) + : : +- CometBroadcastExchange (30) + : : +- CometFilter (29) + : : +- CometScan parquet spark_catalog.default.customer (28) + : +- CometBroadcastExchange (44) + : +- CometProject (43) + : +- CometFilter (42) + : +- CometScan parquet spark_catalog.default.date_dim (41) + +- CometProject (62) + +- CometBroadcastHashJoin (61) + :- CometProject (59) + : +- CometSortMergeJoin (58) + : :- CometSort (52) + : : +- CometColumnarExchange (51) + : : +- CometProject (50) + : : +- CometBroadcastHashJoin (49) + : : :- CometScan parquet spark_catalog.default.web_sales (47) + : : +- ReusedExchange (48) + : +- CometSort (57) + : +- CometProject (56) + : +- CometFilter (55) + : +- CometHashAggregate (54) + : +- ReusedExchange (53) + +- ReusedExchange (60) (1) Scan parquet spark_catalog.default.catalog_sales @@ -75,10 +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 : 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 +(2) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_item_sk#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] @@ -86,203 +83,206 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(4) CometFilter +(3) CometFilter Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] Condition : isnotnull(ss_item_sk#7) -(5) Scan parquet spark_catalog.default.date_dim +(4) 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) CometFilter +(5) 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) CometProject +(6) 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 +(7) CometBroadcastExchange Input [2]: [d_date_sk#10, d_date#11] Arguments: [d_date_sk#10, d_date#11] -(9) CometBroadcastHashJoin +(8) 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 +(9) 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] -(11) Scan parquet spark_catalog.default.item +(10) 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 -(12) CometFilter +(11) CometFilter Input [2]: [i_item_sk#13, i_item_desc#14] Condition : isnotnull(i_item_sk#13) -(13) CometBroadcastExchange +(12) CometBroadcastExchange Input [2]: [i_item_sk#13, i_item_desc#14] Arguments: [i_item_sk#13, i_item_desc#14] -(14) CometBroadcastHashJoin +(13) 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) CometProject +(14) 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) CometHashAggregate +(15) 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)] -(17) ColumnarToRow [codegen id : 1] -Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] - -(18) Exchange +(16) CometColumnarExchange 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] +Arguments: hashpartitioning(_groupingexpression#15, i_item_sk#13, d_date#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(19) HashAggregate [codegen id : 2] +(17) CometHashAggregate 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#13 AS item_sk#18, count(1)#17 AS cnt#19] -(20) Filter [codegen id : 2] -Input [2]: [item_sk#18, cnt#19] -Condition : (cnt#19 > 4) +(18) CometFilter +Input [2]: [item_sk#17, cnt#18] +Condition : (cnt#18 > 4) -(21) Project [codegen id : 2] -Output [1]: [item_sk#18] -Input [2]: [item_sk#18, cnt#19] +(19) CometProject +Input [2]: [item_sk#17, cnt#18] +Arguments: [item_sk#17], [item_sk#17] -(22) BroadcastExchange -Input [1]: [item_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(20) CometBroadcastExchange +Input [1]: [item_sk#17] +Arguments: [item_sk#17] -(23) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#2] -Right keys [1]: [item_sk#18] -Join type: LeftSemi -Join condition: None +(21) CometBroadcastHashJoin +Left output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Right output [1]: [item_sk#17] +Arguments: [cs_item_sk#2], [item_sk#17], LeftSemi, BuildRight -(24) Project [codegen id : 3] -Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +(22) CometProject Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(25) Exchange +(23) CometColumnarExchange 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=3] +Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(26) Sort [codegen id : 4] +(24) CometSort 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 +Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1 ASC NULLS FIRST] -(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] +(25) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(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) +(26) CometFilter +Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] +Condition : isnotnull(ss_customer_sk#19) -(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] +(27) CometProject +Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] +Arguments: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21], [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] -(30) Scan parquet spark_catalog.default.customer -Output [1]: [c_customer_sk#24] +(28) Scan parquet spark_catalog.default.customer +Output [1]: [c_customer_sk#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(31) CometFilter -Input [1]: [c_customer_sk#24] -Condition : isnotnull(c_customer_sk#24) +(29) CometFilter +Input [1]: [c_customer_sk#23] +Condition : isnotnull(c_customer_sk#23) -(32) CometBroadcastExchange -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24] +(30) CometBroadcastExchange +Input [1]: [c_customer_sk#23] +Arguments: [c_customer_sk#23] -(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 +(31) CometBroadcastHashJoin +Left output [3]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] +Right output [1]: [c_customer_sk#23] +Arguments: [ss_customer_sk#19], [c_customer_sk#23], Inner, BuildRight -(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] +(32) CometProject +Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, c_customer_sk#23] +Arguments: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23], [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] + +(33) CometHashAggregate +Input [3]: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] +Keys [1]: [c_customer_sk#23] +Functions [1]: [partial_sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] + +(34) CometColumnarExchange +Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] +Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (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))] - -(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] - -(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))#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 : 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 : 6] -Output [1]: [c_customer_sk#24] -Input [2]: [c_customer_sk#24, ssales#28] - -(41) Sort [codegen id : 6] -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 - -(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 - -(43) Project [codegen id : 8] -Output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] +Keys [1]: [c_customer_sk#23] +Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] + +(36) CometFilter +Input [2]: [c_customer_sk#23, ssales#26] +Condition : (isnotnull(ssales#26) AND (cast(ssales#26 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#27, [id=#28]))) + +(37) CometProject +Input [2]: [c_customer_sk#23, ssales#26] +Arguments: [c_customer_sk#23], [c_customer_sk#23] + +(38) CometSort +Input [1]: [c_customer_sk#23] +Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] + +(39) CometSortMergeJoin +Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Right output [1]: [c_customer_sk#23] +Arguments: [cs_bill_customer_sk#1], [c_customer_sk#23], LeftSemi + +(40) CometProject Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Arguments: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] + +(41) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#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,2), IsNotNull(d_date_sk)] +ReadSchema: struct + +(42) CometFilter +Input [3]: [d_date_sk#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 = 2)) AND isnotnull(d_date_sk#29)) + +(43) CometProject +Input [3]: [d_date_sk#29, d_year#30, d_moy#31] +Arguments: [d_date_sk#29], [d_date_sk#29] -(44) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#31] +(44) CometBroadcastExchange +Input [1]: [d_date_sk#29] +Arguments: [d_date_sk#29] -(45) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#31] -Join type: Inner -Join condition: None +(45) CometBroadcastHashJoin +Left output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Right output [1]: [d_date_sk#29] +Arguments: [cs_sold_date_sk#5], [d_date_sk#29], Inner, BuildRight -(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) CometProject +Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#29] +Arguments: [sales#32], [(cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4) AS sales#32] (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] @@ -291,94 +291,87 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#37), dynamicpruningexpression(ws_sold_date_sk#37 IN dynamicpruning#38)] ReadSchema: struct -(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] - -(49) ReusedExchange [Reuses operator id: 22] +(48) ReusedExchange [Reuses operator id: 20] Output [1]: [item_sk#39] -(50) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ws_item_sk#33] -Right keys [1]: [item_sk#39] -Join type: LeftSemi -Join condition: None +(49) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#33, ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] +Right output [1]: [item_sk#39] +Arguments: [ws_item_sk#33], [item_sk#39], LeftSemi, BuildRight -(51) Project [codegen id : 11] -Output [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] +(50) CometProject Input [5]: [ws_item_sk#33, ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] +Arguments: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37], [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -(52) Exchange +(51) CometColumnarExchange 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] +Arguments: hashpartitioning(ws_bill_customer_sk#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(53) Sort [codegen id : 12] +(52) CometSort 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 +Arguments: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37], [ws_bill_customer_sk#34 ASC NULLS FIRST] -(54) ReusedExchange [Reuses operator id: 37] +(53) ReusedExchange [Reuses operator id: 34] Output [3]: [c_customer_sk#40, sum#41, isEmpty#42] -(55) HashAggregate [codegen id : 14] +(54) CometHashAggregate Input [3]: [c_customer_sk#40, sum#41, isEmpty#42] Keys [1]: [c_customer_sk#40] Functions [1]: [sum((cast(ss_quantity#43 as decimal(10,0)) * ss_sales_price#44))] -Aggregate Attributes [1]: [sum((cast(ss_quantity#43 as decimal(10,0)) * ss_sales_price#44))#27] -Results [2]: [c_customer_sk#40, sum((cast(ss_quantity#43 as decimal(10,0)) * ss_sales_price#44))#27 AS ssales#45] -(56) Filter [codegen id : 14] +(55) CometFilter Input [2]: [c_customer_sk#40, ssales#45] -Condition : (isnotnull(ssales#45) AND (cast(ssales#45 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#29, [id=#30]))) +Condition : (isnotnull(ssales#45) AND (cast(ssales#45 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#27, [id=#28]))) -(57) Project [codegen id : 14] -Output [1]: [c_customer_sk#40] +(56) CometProject Input [2]: [c_customer_sk#40, ssales#45] +Arguments: [c_customer_sk#40], [c_customer_sk#40] -(58) Sort [codegen id : 14] +(57) CometSort Input [1]: [c_customer_sk#40] -Arguments: [c_customer_sk#40 ASC NULLS FIRST], false, 0 +Arguments: [c_customer_sk#40], [c_customer_sk#40 ASC NULLS FIRST] -(59) SortMergeJoin [codegen id : 16] -Left keys [1]: [ws_bill_customer_sk#34] -Right keys [1]: [c_customer_sk#40] -Join type: LeftSemi -Join condition: None +(58) CometSortMergeJoin +Left output [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] +Right output [1]: [c_customer_sk#40] +Arguments: [ws_bill_customer_sk#34], [c_customer_sk#40], LeftSemi -(60) Project [codegen id : 16] -Output [3]: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] +(59) CometProject Input [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] +Arguments: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37], [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -(61) ReusedExchange [Reuses operator id: 72] +(60) ReusedExchange [Reuses operator id: 44] Output [1]: [d_date_sk#46] -(62) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ws_sold_date_sk#37] -Right keys [1]: [d_date_sk#46] -Join type: Inner -Join condition: None +(61) CometBroadcastHashJoin +Left output [3]: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] +Right output [1]: [d_date_sk#46] +Arguments: [ws_sold_date_sk#37], [d_date_sk#46], Inner, BuildRight -(63) Project [codegen id : 16] -Output [1]: [(cast(ws_quantity#35 as decimal(10,0)) * ws_list_price#36) AS sales#47] +(62) CometProject Input [4]: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37, d_date_sk#46] +Arguments: [sales#47], [(cast(ws_quantity#35 as decimal(10,0)) * ws_list_price#36) AS sales#47] -(64) Union +(63) CometUnion +Child 0 Input [1]: [sales#32] +Child 1 Input [1]: [sales#47] -(65) HashAggregate [codegen id : 17] +(64) CometHashAggregate Input [1]: [sales#32] Keys: [] Functions [1]: [partial_sum(sales#32)] -Aggregate Attributes [2]: [sum#48, isEmpty#49] -Results [2]: [sum#50, isEmpty#51] -(66) Exchange -Input [2]: [sum#50, isEmpty#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +(65) CometColumnarExchange +Input [2]: [sum#48, isEmpty#49] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(67) HashAggregate [codegen id : 18] -Input [2]: [sum#50, isEmpty#51] +(66) CometHashAggregate +Input [2]: [sum#48, isEmpty#49] Keys: [] Functions [1]: [sum(sales#32)] -Aggregate Attributes [1]: [sum(sales#32)#52] -Results [1]: [sum(sales#32)#52 AS sum(sales)#53] + +(67) ColumnarToRow [codegen id : 1] +Input [1]: [sum(sales)#50] ===== Subqueries ===== @@ -391,28 +384,28 @@ BroadcastExchange (72) (68) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#31, d_year#54, d_moy#55] +Output [3]: [d_date_sk#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,2), IsNotNull(d_date_sk)] ReadSchema: struct (69) CometFilter -Input [3]: [d_date_sk#31, d_year#54, d_moy#55] -Condition : ((((isnotnull(d_year#54) AND isnotnull(d_moy#55)) AND (d_year#54 = 2000)) AND (d_moy#55 = 2)) AND isnotnull(d_date_sk#31)) +Input [3]: [d_date_sk#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 = 2)) AND isnotnull(d_date_sk#29)) (70) CometProject -Input [3]: [d_date_sk#31, d_year#54, d_moy#55] -Arguments: [d_date_sk#31], [d_date_sk#31] +Input [3]: [d_date_sk#29, d_year#30, d_moy#31] +Arguments: [d_date_sk#29], [d_date_sk#29] (71) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#31] +Input [1]: [d_date_sk#29] (72) BroadcastExchange -Input [1]: [d_date_sk#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Input [1]: [d_date_sk#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 2 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (77) +- * ColumnarToRow (76) +- CometProject (75) @@ -440,15 +433,15 @@ Input [2]: [d_date_sk#10, d_date#11] (77) BroadcastExchange Input [2]: [d_date_sk#10, d_date#11] -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) +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] + +Subquery:3 Hosting operator id = 36 Hosting Expression = Subquery scalar-subquery#27, [id=#28] +* ColumnarToRow (95) ++- CometHashAggregate (94) + +- CometColumnarExchange (93) + +- CometHashAggregate (92) + +- CometHashAggregate (91) + +- CometColumnarExchange (90) +- CometHashAggregate (89) +- CometProject (88) +- CometBroadcastHashJoin (87) @@ -464,95 +457,89 @@ Subquery:3 Hosting operator id = 39 Hosting Expression = Subquery scalar-subquer (78) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#56, ss_quantity#57, ss_sales_price#58, ss_sold_date_sk#59] +Output [4]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54] 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#54), dynamicpruningexpression(ss_sold_date_sk#54 IN dynamicpruning#55)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (79) CometFilter -Input [4]: [ss_customer_sk#56, ss_quantity#57, ss_sales_price#58, ss_sold_date_sk#59] -Condition : isnotnull(ss_customer_sk#56) +Input [4]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54] +Condition : isnotnull(ss_customer_sk#51) -(80) ReusedExchange [Reuses operator id: 32] -Output [1]: [c_customer_sk#61] +(80) ReusedExchange [Reuses operator id: 30] +Output [1]: [c_customer_sk#56] (81) CometBroadcastHashJoin -Left output [4]: [ss_customer_sk#56, ss_quantity#57, ss_sales_price#58, ss_sold_date_sk#59] -Right output [1]: [c_customer_sk#61] -Arguments: [ss_customer_sk#56], [c_customer_sk#61], Inner, BuildRight +Left output [4]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54] +Right output [1]: [c_customer_sk#56] +Arguments: [ss_customer_sk#51], [c_customer_sk#56], Inner, BuildRight (82) CometProject -Input [5]: [ss_customer_sk#56, ss_quantity#57, ss_sales_price#58, ss_sold_date_sk#59, c_customer_sk#61] -Arguments: [ss_quantity#57, ss_sales_price#58, ss_sold_date_sk#59, c_customer_sk#61], [ss_quantity#57, ss_sales_price#58, ss_sold_date_sk#59, c_customer_sk#61] +Input [5]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56] +Arguments: [ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56], [ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56] (83) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#62, d_year#63] +Output [2]: [d_date_sk#57, d_year#58] 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) CometFilter -Input [2]: [d_date_sk#62, d_year#63] -Condition : (d_year#63 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#62)) +Input [2]: [d_date_sk#57, d_year#58] +Condition : (d_year#58 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#57)) (85) CometProject -Input [2]: [d_date_sk#62, d_year#63] -Arguments: [d_date_sk#62], [d_date_sk#62] +Input [2]: [d_date_sk#57, d_year#58] +Arguments: [d_date_sk#57], [d_date_sk#57] (86) CometBroadcastExchange -Input [1]: [d_date_sk#62] -Arguments: [d_date_sk#62] +Input [1]: [d_date_sk#57] +Arguments: [d_date_sk#57] (87) CometBroadcastHashJoin -Left output [4]: [ss_quantity#57, ss_sales_price#58, ss_sold_date_sk#59, c_customer_sk#61] -Right output [1]: [d_date_sk#62] -Arguments: [ss_sold_date_sk#59], [d_date_sk#62], Inner, BuildRight +Left output [4]: [ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56] +Right output [1]: [d_date_sk#57] +Arguments: [ss_sold_date_sk#54], [d_date_sk#57], Inner, BuildRight (88) CometProject -Input [5]: [ss_quantity#57, ss_sales_price#58, ss_sold_date_sk#59, c_customer_sk#61, d_date_sk#62] -Arguments: [ss_quantity#57, ss_sales_price#58, c_customer_sk#61], [ss_quantity#57, ss_sales_price#58, c_customer_sk#61] +Input [5]: [ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56, d_date_sk#57] +Arguments: [ss_quantity#52, ss_sales_price#53, c_customer_sk#56], [ss_quantity#52, ss_sales_price#53, c_customer_sk#56] (89) CometHashAggregate -Input [3]: [ss_quantity#57, ss_sales_price#58, c_customer_sk#61] -Keys [1]: [c_customer_sk#61] -Functions [1]: [partial_sum((cast(ss_quantity#57 as decimal(10,0)) * ss_sales_price#58))] - -(90) ColumnarToRow [codegen id : 1] -Input [3]: [c_customer_sk#61, sum#64, isEmpty#65] - -(91) Exchange -Input [3]: [c_customer_sk#61, sum#64, isEmpty#65] -Arguments: hashpartitioning(c_customer_sk#61, 5), ENSURE_REQUIREMENTS, [plan_id=9] - -(92) HashAggregate [codegen id : 2] -Input [3]: [c_customer_sk#61, sum#64, isEmpty#65] -Keys [1]: [c_customer_sk#61] -Functions [1]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_sales_price#58))] -Aggregate Attributes [1]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_sales_price#58))#66] -Results [1]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_sales_price#58))#66 AS csales#67] - -(93) HashAggregate [codegen id : 2] -Input [1]: [csales#67] +Input [3]: [ss_quantity#52, ss_sales_price#53, c_customer_sk#56] +Keys [1]: [c_customer_sk#56] +Functions [1]: [partial_sum((cast(ss_quantity#52 as decimal(10,0)) * ss_sales_price#53))] + +(90) CometColumnarExchange +Input [3]: [c_customer_sk#56, sum#59, isEmpty#60] +Arguments: hashpartitioning(c_customer_sk#56, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(91) CometHashAggregate +Input [3]: [c_customer_sk#56, sum#59, isEmpty#60] +Keys [1]: [c_customer_sk#56] +Functions [1]: [sum((cast(ss_quantity#52 as decimal(10,0)) * ss_sales_price#53))] + +(92) CometHashAggregate +Input [1]: [csales#61] Keys: [] -Functions [1]: [partial_max(csales#67)] -Aggregate Attributes [1]: [max#68] -Results [1]: [max#69] +Functions [1]: [partial_max(csales#61)] -(94) Exchange -Input [1]: [max#69] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] +(93) CometColumnarExchange +Input [1]: [max#62] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(95) HashAggregate [codegen id : 3] -Input [1]: [max#69] +(94) CometHashAggregate +Input [1]: [max#62] Keys: [] -Functions [1]: [max(csales#67)] -Aggregate Attributes [1]: [max(csales#67)#70] -Results [1]: [max(csales#67)#70 AS tpcds_cmax#71] +Functions [1]: [max(csales#61)] + +(95) ColumnarToRow [codegen id : 1] +Input [1]: [tpcds_cmax#63] -Subquery:4 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#59 IN dynamicpruning#60 +Subquery:4 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#54 IN dynamicpruning#55 BroadcastExchange (100) +- * ColumnarToRow (99) +- CometProject (98) @@ -561,29 +548,29 @@ BroadcastExchange (100) (96) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#62, d_year#63] +Output [2]: [d_date_sk#57, d_year#58] 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 (97) CometFilter -Input [2]: [d_date_sk#62, d_year#63] -Condition : (d_year#63 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#62)) +Input [2]: [d_date_sk#57, d_year#58] +Condition : (d_year#58 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#57)) (98) CometProject -Input [2]: [d_date_sk#62, d_year#63] -Arguments: [d_date_sk#62], [d_date_sk#62] +Input [2]: [d_date_sk#57, d_year#58] +Arguments: [d_date_sk#57], [d_date_sk#57] (99) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#62] +Input [1]: [d_date_sk#57] (100) BroadcastExchange -Input [1]: [d_date_sk#62] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] +Input [1]: [d_date_sk#57] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] Subquery:5 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#37 IN dynamicpruning#6 -Subquery:6 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquery scalar-subquery#29, [id=#30] +Subquery:6 Hosting operator id = 55 Hosting Expression = ReusedSubquery Subquery scalar-subquery#27, [id=#28] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/simplified.txt index e065a0a87..18cf2b2c6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/simplified.txt @@ -1,150 +1,116 @@ -WholeStageCodegen (18) - HashAggregate [sum,isEmpty] [sum(sales),sum(sales),sum,isEmpty] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (17) - HashAggregate [sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - 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 (4) - Sort [cs_bill_customer_sk] - InputAdapter - Exchange [cs_bill_customer_sk] #2 - WholeStageCodegen (3) - Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - BroadcastHashJoin [cs_item_sk,item_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #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 + CometHashAggregate [sum(sales),sum,isEmpty,sum(sales)] + CometColumnarExchange #1 + CometHashAggregate [sum,isEmpty,sales] + CometUnion [sales] + CometProject [cs_quantity,cs_list_price] [sales] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] + CometProject [cs_quantity,cs_list_price,cs_sold_date_sk] + CometSortMergeJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk] + CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + CometColumnarExchange [cs_bill_customer_sk] #2 + CometProject [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [item_sk] #4 + CometProject [item_sk] + CometFilter [item_sk,cnt] + CometHashAggregate [item_sk,cnt,_groupingexpression,i_item_sk,d_date,count,count(1)] + CometColumnarExchange [_groupingexpression,i_item_sk,d_date] #5 + CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] + CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] + CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] + CometProject [ss_item_sk,d_date] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #6 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] - CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] - CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [i_item_sk,i_item_desc] #8 - CometFilter [i_item_sk,i_item_desc] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - InputAdapter - WholeStageCodegen (6) - Sort [c_customer_sk] - Project [c_customer_sk] - Filter [ssales] - Subquery #3 - WholeStageCodegen (3) - HashAggregate [max] [max(csales),tpcds_cmax,max] - InputAdapter - 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] #12 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] - CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] - CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_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 #13 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange [d_date_sk] #14 + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + CometBroadcastExchange [d_date_sk,d_date] #7 + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + CometBroadcastExchange [i_item_sk,i_item_desc] #8 + CometFilter [i_item_sk,i_item_desc] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + CometSort [c_customer_sk] + CometProject [c_customer_sk] + CometFilter [c_customer_sk,ssales] + Subquery #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [tpcds_cmax,max,max(csales)] + CometColumnarExchange #11 + CometHashAggregate [max,csales] + CometHashAggregate [csales,c_customer_sk,sum,isEmpty,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + CometColumnarExchange [c_customer_sk] #12 + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] + CometProject [ss_quantity,ss_sales_price,c_customer_sk] + CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] + CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_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 #13 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] 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] #9 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] - CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] - CometProject [ss_customer_sk,ss_quantity,ss_sales_price] - CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [c_customer_sk] #10 - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk] - InputAdapter - ReusedExchange [d_date_sk] #3 - 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 (12) - Sort [ws_bill_customer_sk] - InputAdapter - 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 - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [item_sk] #4 - InputAdapter - 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] #9 - InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [c_customer_sk] #10 + CometBroadcastExchange [d_date_sk] #14 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometHashAggregate [c_customer_sk,ssales,sum,isEmpty,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + CometColumnarExchange [c_customer_sk] #9 + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] + CometProject [ss_quantity,ss_sales_price,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] + CometProject [ss_customer_sk,ss_quantity,ss_sales_price] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometBroadcastExchange [c_customer_sk] #10 + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk] + CometBroadcastExchange [d_date_sk] #15 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [ws_quantity,ws_list_price] [sales] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] + CometProject [ws_quantity,ws_list_price,ws_sold_date_sk] + CometSortMergeJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk] + CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + CometColumnarExchange [ws_bill_customer_sk] #16 + CometProject [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [item_sk] #4 + CometSort [c_customer_sk] + CometProject [c_customer_sk] + CometFilter [c_customer_sk,ssales] + ReusedSubquery [tpcds_cmax] #3 + CometHashAggregate [c_customer_sk,ssales,sum,isEmpty,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + ReusedExchange [c_customer_sk,sum,isEmpty] #9 + ReusedExchange [d_date_sk] #15 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/explain.txt index 1b35c983a..ee11e7897 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/explain.txt @@ -1,92 +1,91 @@ == Physical Plan == -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 (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) +* ColumnarToRow (87) ++- CometTakeOrderedAndProject (86) + +- CometUnion (85) + :- CometHashAggregate (62) + : +- CometColumnarExchange (61) + : +- CometHashAggregate (60) + : +- CometProject (59) + : +- CometBroadcastHashJoin (58) + : :- CometProject (53) + : : +- CometBroadcastHashJoin (52) + : : :- CometSortMergeJoin (40) + : : : :- CometSort (25) + : : : : +- CometColumnarExchange (24) + : : : : +- CometProject (23) + : : : : +- CometBroadcastHashJoin (22) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : +- CometBroadcastExchange (21) + : : : : +- CometProject (20) + : : : : +- CometFilter (19) + : : : : +- CometHashAggregate (18) + : : : : +- CometColumnarExchange (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) + : : : +- CometSort (39) + : : : +- CometProject (38) + : : : +- CometFilter (37) + : : : +- CometHashAggregate (36) + : : : +- CometColumnarExchange (35) + : : : +- CometHashAggregate (34) + : : : +- CometProject (33) + : : : +- CometBroadcastHashJoin (32) + : : : :- CometProject (28) + : : : : +- CometFilter (27) + : : : : +- CometScan parquet spark_catalog.default.store_sales (26) + : : : +- CometBroadcastExchange (31) + : : : +- CometFilter (30) + : : : +- CometScan parquet spark_catalog.default.customer (29) + : : +- CometBroadcastExchange (51) + : : +- CometSortMergeJoin (50) + : : :- CometSort (44) + : : : +- CometColumnarExchange (43) + : : : +- CometFilter (42) + : : : +- CometScan parquet spark_catalog.default.customer (41) + : : +- CometSort (49) + : : +- CometProject (48) + : : +- CometFilter (47) + : : +- CometHashAggregate (46) + : : +- ReusedExchange (45) + : +- CometBroadcastExchange (57) + : +- CometProject (56) + : +- CometFilter (55) + : +- CometScan parquet spark_catalog.default.date_dim (54) + +- CometHashAggregate (84) + +- CometColumnarExchange (83) + +- CometHashAggregate (82) + +- CometProject (81) + +- CometBroadcastHashJoin (80) + :- CometProject (78) + : +- CometBroadcastHashJoin (77) + : :- CometSortMergeJoin (75) + : : :- CometSort (69) + : : : +- CometColumnarExchange (68) + : : : +- CometProject (67) + : : : +- CometBroadcastHashJoin (66) + : : : :- CometFilter (64) + : : : : +- CometScan parquet spark_catalog.default.web_sales (63) + : : : +- ReusedExchange (65) + : : +- CometSort (74) + : : +- CometProject (73) + : : +- CometFilter (72) + : : +- CometHashAggregate (71) + : : +- ReusedExchange (70) + : +- ReusedExchange (76) + +- ReusedExchange (79) (1) Scan parquet spark_catalog.default.catalog_sales @@ -101,10 +100,7 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] Condition : isnotnull(ss_item_sk#7) -(6) Scan parquet spark_catalog.default.date_dim +(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 -(7) CometFilter +(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)) -(8) CometProject +(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] -(9) CometBroadcastExchange +(8) CometBroadcastExchange Input [2]: [d_date_sk#10, d_date#11] Arguments: [d_date_sk#10, d_date#11] -(10) CometBroadcastHashJoin +(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 -(11) CometProject +(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] -(12) Scan parquet spark_catalog.default.item +(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 -(13) CometFilter +(12) CometFilter Input [2]: [i_item_sk#13, i_item_desc#14] Condition : isnotnull(i_item_sk#13) -(14) CometBroadcastExchange +(13) CometBroadcastExchange Input [2]: [i_item_sk#13, i_item_desc#14] Arguments: [i_item_sk#13, i_item_desc#14] -(15) CometBroadcastHashJoin +(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 -(16) CometProject +(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] -(17) CometHashAggregate +(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)] -(18) ColumnarToRow [codegen id : 1] -Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] - -(19) Exchange +(17) CometColumnarExchange 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] +Arguments: hashpartitioning(_groupingexpression#15, i_item_sk#13, d_date#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(20) HashAggregate [codegen id : 2] +(18) CometHashAggregate 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#13 AS item_sk#18, count(1)#17 AS cnt#19] -(21) Filter [codegen id : 2] -Input [2]: [item_sk#18, cnt#19] -Condition : (cnt#19 > 4) +(19) CometFilter +Input [2]: [item_sk#17, cnt#18] +Condition : (cnt#18 > 4) -(22) Project [codegen id : 2] -Output [1]: [item_sk#18] -Input [2]: [item_sk#18, cnt#19] +(20) CometProject +Input [2]: [item_sk#17, cnt#18] +Arguments: [item_sk#17], [item_sk#17] -(23) BroadcastExchange -Input [1]: [item_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(21) CometBroadcastExchange +Input [1]: [item_sk#17] +Arguments: [item_sk#17] -(24) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#2] -Right keys [1]: [item_sk#18] -Join type: LeftSemi -Join condition: None +(22) CometBroadcastHashJoin +Left output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Right output [1]: [item_sk#17] +Arguments: [cs_item_sk#2], [item_sk#17], LeftSemi, BuildRight -(25) Project [codegen id : 3] -Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +(23) CometProject Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(26) Exchange +(24) CometColumnarExchange 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=3] +Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(27) Sort [codegen id : 4] +(25) CometSort 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 +Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1 ASC NULLS FIRST] -(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] +(26) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(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) +(27) CometFilter +Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] +Condition : isnotnull(ss_customer_sk#19) -(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] +(28) CometProject +Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] +Arguments: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21], [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] -(31) Scan parquet spark_catalog.default.customer -Output [1]: [c_customer_sk#24] +(29) Scan parquet spark_catalog.default.customer +Output [1]: [c_customer_sk#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(32) CometFilter -Input [1]: [c_customer_sk#24] -Condition : isnotnull(c_customer_sk#24) +(30) CometFilter +Input [1]: [c_customer_sk#23] +Condition : isnotnull(c_customer_sk#23) -(33) CometBroadcastExchange -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24] +(31) CometBroadcastExchange +Input [1]: [c_customer_sk#23] +Arguments: [c_customer_sk#23] -(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 +(32) CometBroadcastHashJoin +Left output [3]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] +Right output [1]: [c_customer_sk#23] +Arguments: [ss_customer_sk#19], [c_customer_sk#23], Inner, BuildRight -(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] +(33) CometProject +Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, c_customer_sk#23] +Arguments: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23], [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] + +(34) CometHashAggregate +Input [3]: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] +Keys [1]: [c_customer_sk#23] +Functions [1]: [partial_sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] + +(35) CometColumnarExchange +Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] +Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (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))] - -(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] - -(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))#27] -Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#27 AS ssales#28] - -(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]))) - -(41) Project [codegen id : 6] -Output [1]: [c_customer_sk#24] -Input [2]: [c_customer_sk#24, ssales#28] - -(42) Sort [codegen id : 6] -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 - -(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 - -(44) Scan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33] +Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] +Keys [1]: [c_customer_sk#23] +Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] + +(37) CometFilter +Input [2]: [c_customer_sk#23, ssales#26] +Condition : (isnotnull(ssales#26) AND (cast(ssales#26 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#27, [id=#28]))) + +(38) CometProject +Input [2]: [c_customer_sk#23, ssales#26] +Arguments: [c_customer_sk#23], [c_customer_sk#23] + +(39) CometSort +Input [1]: [c_customer_sk#23] +Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] + +(40) CometSortMergeJoin +Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Right output [1]: [c_customer_sk#23] +Arguments: [cs_bill_customer_sk#1], [c_customer_sk#23], LeftSemi + +(41) Scan parquet spark_catalog.default.customer +Output [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(45) CometFilter -Input [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33] -Condition : isnotnull(c_customer_sk#31) - -(46) ColumnarToRow [codegen id : 7] -Input [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33] - -(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] - -(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 - -(49) ReusedExchange [Reuses operator id: 38] -Output [3]: [c_customer_sk#24, sum#25, isEmpty#26] - -(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))#27] -Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#27 AS ssales#28] - -(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]))) - -(52) Project [codegen id : 10] -Output [1]: [c_customer_sk#24] -Input [2]: [c_customer_sk#24, ssales#28] - -(53) Sort [codegen id : 10] -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 - -(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 - -(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] - -(56) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#31] -Join type: Inner -Join condition: None - -(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] - -(58) ReusedExchange [Reuses operator id: 93] -Output [1]: [d_date_sk#34] - -(59) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#34] -Join type: Inner -Join condition: None - -(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] - -(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] +(42) CometFilter +Input [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] +Condition : isnotnull(c_customer_sk#29) + +(43) CometColumnarExchange +Input [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] +Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(44) CometSort +Input [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] +Arguments: [c_customer_sk#29, c_first_name#30, c_last_name#31], [c_customer_sk#29 ASC NULLS FIRST] + +(45) ReusedExchange [Reuses operator id: 35] +Output [3]: [c_customer_sk#23, sum#24, isEmpty#25] + +(46) CometHashAggregate +Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] +Keys [1]: [c_customer_sk#23] +Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] + +(47) CometFilter +Input [2]: [c_customer_sk#23, ssales#26] +Condition : (isnotnull(ssales#26) AND (cast(ssales#26 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#27, [id=#28]))) + +(48) CometProject +Input [2]: [c_customer_sk#23, ssales#26] +Arguments: [c_customer_sk#23], [c_customer_sk#23] + +(49) CometSort +Input [1]: [c_customer_sk#23] +Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] + +(50) CometSortMergeJoin +Left output [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] +Right output [1]: [c_customer_sk#23] +Arguments: [c_customer_sk#29], [c_customer_sk#23], LeftSemi + +(51) CometBroadcastExchange +Input [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] +Arguments: [c_customer_sk#29, c_first_name#30, c_last_name#31] + +(52) CometBroadcastHashJoin +Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Right output [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] +Arguments: [cs_bill_customer_sk#1], [c_customer_sk#29], Inner, BuildRight + +(53) CometProject +Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#29, c_first_name#30, c_last_name#31] +Arguments: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#30, c_last_name#31], [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#30, c_last_name#31] + +(54) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#32, d_year#33, d_moy#34] +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 + +(55) CometFilter +Input [3]: [d_date_sk#32, d_year#33, d_moy#34] +Condition : ((((isnotnull(d_year#33) AND isnotnull(d_moy#34)) AND (d_year#33 = 2000)) AND (d_moy#34 = 2)) AND isnotnull(d_date_sk#32)) + +(56) CometProject +Input [3]: [d_date_sk#32, d_year#33, d_moy#34] +Arguments: [d_date_sk#32], [d_date_sk#32] + +(57) CometBroadcastExchange +Input [1]: [d_date_sk#32] +Arguments: [d_date_sk#32] + +(58) CometBroadcastHashJoin +Left output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#30, c_last_name#31] +Right output [1]: [d_date_sk#32] +Arguments: [cs_sold_date_sk#5], [d_date_sk#32], Inner, BuildRight + +(59) CometProject +Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#30, c_last_name#31, d_date_sk#32] +Arguments: [cs_quantity#3, cs_list_price#4, c_first_name#30, c_last_name#31], [cs_quantity#3, cs_list_price#4, c_first_name#30, c_last_name#31] + +(60) CometHashAggregate +Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#30, c_last_name#31] +Keys [2]: [c_last_name#31, c_first_name#30] Functions [1]: [partial_sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] -Aggregate Attributes [2]: [sum#35, isEmpty#36] -Results [4]: [c_last_name#33, c_first_name#32, sum#37, isEmpty#38] -(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] +(61) CometColumnarExchange +Input [4]: [c_last_name#31, c_first_name#30, sum#35, isEmpty#36] +Arguments: hashpartitioning(c_last_name#31, c_first_name#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(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] +(62) CometHashAggregate +Input [4]: [c_last_name#31, c_first_name#30, sum#35, isEmpty#36] +Keys [2]: [c_last_name#31, c_first_name#30] 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))#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] -(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] +(63) Scan parquet spark_catalog.default.web_sales +Output [5]: [ws_item_sk#37, ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#45), dynamicpruningexpression(ws_sold_date_sk#45 IN dynamicpruning#46)] +PartitionFilters: [isnotnull(ws_sold_date_sk#41), dynamicpruningexpression(ws_sold_date_sk#41 IN dynamicpruning#42)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(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) - -(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] - -(67) ReusedExchange [Reuses operator id: 23] -Output [1]: [item_sk#47] - -(68) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#41] -Right keys [1]: [item_sk#47] -Join type: LeftSemi -Join condition: None - -(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] - -(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] - -(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 - -(72) ReusedExchange [Reuses operator id: 38] -Output [3]: [c_customer_sk#48, sum#49, isEmpty#50] - -(73) HashAggregate [codegen id : 20] -Input [3]: [c_customer_sk#48, sum#49, isEmpty#50] -Keys [1]: [c_customer_sk#48] -Functions [1]: [sum((cast(ss_quantity#51 as decimal(10,0)) * ss_sales_price#52))] -Aggregate Attributes [1]: [sum((cast(ss_quantity#51 as decimal(10,0)) * ss_sales_price#52))#27] -Results [2]: [c_customer_sk#48, sum((cast(ss_quantity#51 as decimal(10,0)) * ss_sales_price#52))#27 AS ssales#53] - -(74) Filter [codegen id : 20] -Input [2]: [c_customer_sk#48, ssales#53] -Condition : (isnotnull(ssales#53) AND (cast(ssales#53 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#29, [id=#30]))) - -(75) Project [codegen id : 20] -Output [1]: [c_customer_sk#48] -Input [2]: [c_customer_sk#48, ssales#53] - -(76) Sort [codegen id : 20] -Input [1]: [c_customer_sk#48] -Arguments: [c_customer_sk#48 ASC NULLS FIRST], false, 0 - -(77) SortMergeJoin [codegen id : 27] -Left keys [1]: [ws_bill_customer_sk#42] -Right keys [1]: [c_customer_sk#48] -Join type: LeftSemi -Join condition: None - -(78) ReusedExchange [Reuses operator id: 55] -Output [3]: [c_customer_sk#54, c_first_name#55, c_last_name#56] - -(79) BroadcastHashJoin [codegen id : 27] -Left keys [1]: [ws_bill_customer_sk#42] -Right keys [1]: [c_customer_sk#54] -Join type: Inner -Join condition: None - -(80) Project [codegen id : 27] -Output [5]: [ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45, c_first_name#55, c_last_name#56] -Input [7]: [ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45, c_customer_sk#54, c_first_name#55, c_last_name#56] - -(81) ReusedExchange [Reuses operator id: 93] -Output [1]: [d_date_sk#57] - -(82) BroadcastHashJoin [codegen id : 27] -Left keys [1]: [ws_sold_date_sk#45] -Right keys [1]: [d_date_sk#57] -Join type: Inner -Join condition: None - -(83) Project [codegen id : 27] -Output [4]: [ws_quantity#43, ws_list_price#44, c_first_name#55, c_last_name#56] -Input [6]: [ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45, c_first_name#55, c_last_name#56, d_date_sk#57] - -(84) HashAggregate [codegen id : 27] -Input [4]: [ws_quantity#43, ws_list_price#44, c_first_name#55, c_last_name#56] -Keys [2]: [c_last_name#56, c_first_name#55] -Functions [1]: [partial_sum((cast(ws_quantity#43 as decimal(10,0)) * ws_list_price#44))] -Aggregate Attributes [2]: [sum#58, isEmpty#59] -Results [4]: [c_last_name#56, c_first_name#55, sum#60, isEmpty#61] - -(85) Exchange -Input [4]: [c_last_name#56, c_first_name#55, sum#60, isEmpty#61] -Arguments: hashpartitioning(c_last_name#56, c_first_name#55, 5), ENSURE_REQUIREMENTS, [plan_id=9] - -(86) HashAggregate [codegen id : 28] -Input [4]: [c_last_name#56, c_first_name#55, sum#60, isEmpty#61] -Keys [2]: [c_last_name#56, c_first_name#55] -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))#62] -Results [3]: [c_last_name#56, c_first_name#55, sum((cast(ws_quantity#43 as decimal(10,0)) * ws_list_price#44))#62 AS sales#63] - -(87) Union - -(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] +(64) CometFilter +Input [5]: [ws_item_sk#37, ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] +Condition : isnotnull(ws_bill_customer_sk#38) + +(65) ReusedExchange [Reuses operator id: 21] +Output [1]: [item_sk#43] + +(66) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#37, ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] +Right output [1]: [item_sk#43] +Arguments: [ws_item_sk#37], [item_sk#43], LeftSemi, BuildRight + +(67) CometProject +Input [5]: [ws_item_sk#37, ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] +Arguments: [ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41], [ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] + +(68) CometColumnarExchange +Input [4]: [ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] +Arguments: hashpartitioning(ws_bill_customer_sk#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(69) CometSort +Input [4]: [ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] +Arguments: [ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41], [ws_bill_customer_sk#38 ASC NULLS FIRST] + +(70) ReusedExchange [Reuses operator id: 35] +Output [3]: [c_customer_sk#44, sum#45, isEmpty#46] + +(71) CometHashAggregate +Input [3]: [c_customer_sk#44, sum#45, isEmpty#46] +Keys [1]: [c_customer_sk#44] +Functions [1]: [sum((cast(ss_quantity#47 as decimal(10,0)) * ss_sales_price#48))] + +(72) CometFilter +Input [2]: [c_customer_sk#44, ssales#49] +Condition : (isnotnull(ssales#49) AND (cast(ssales#49 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#27, [id=#28]))) + +(73) CometProject +Input [2]: [c_customer_sk#44, ssales#49] +Arguments: [c_customer_sk#44], [c_customer_sk#44] + +(74) CometSort +Input [1]: [c_customer_sk#44] +Arguments: [c_customer_sk#44], [c_customer_sk#44 ASC NULLS FIRST] + +(75) CometSortMergeJoin +Left output [4]: [ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] +Right output [1]: [c_customer_sk#44] +Arguments: [ws_bill_customer_sk#38], [c_customer_sk#44], LeftSemi + +(76) ReusedExchange [Reuses operator id: 51] +Output [3]: [c_customer_sk#50, c_first_name#51, c_last_name#52] + +(77) CometBroadcastHashJoin +Left output [4]: [ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] +Right output [3]: [c_customer_sk#50, c_first_name#51, c_last_name#52] +Arguments: [ws_bill_customer_sk#38], [c_customer_sk#50], Inner, BuildRight + +(78) CometProject +Input [7]: [ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41, c_customer_sk#50, c_first_name#51, c_last_name#52] +Arguments: [ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41, c_first_name#51, c_last_name#52], [ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41, c_first_name#51, c_last_name#52] + +(79) ReusedExchange [Reuses operator id: 57] +Output [1]: [d_date_sk#53] + +(80) CometBroadcastHashJoin +Left output [5]: [ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41, c_first_name#51, c_last_name#52] +Right output [1]: [d_date_sk#53] +Arguments: [ws_sold_date_sk#41], [d_date_sk#53], Inner, BuildRight + +(81) CometProject +Input [6]: [ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41, c_first_name#51, c_last_name#52, d_date_sk#53] +Arguments: [ws_quantity#39, ws_list_price#40, c_first_name#51, c_last_name#52], [ws_quantity#39, ws_list_price#40, c_first_name#51, c_last_name#52] + +(82) CometHashAggregate +Input [4]: [ws_quantity#39, ws_list_price#40, c_first_name#51, c_last_name#52] +Keys [2]: [c_last_name#52, c_first_name#51] +Functions [1]: [partial_sum((cast(ws_quantity#39 as decimal(10,0)) * ws_list_price#40))] + +(83) CometColumnarExchange +Input [4]: [c_last_name#52, c_first_name#51, sum#54, isEmpty#55] +Arguments: hashpartitioning(c_last_name#52, c_first_name#51, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(84) CometHashAggregate +Input [4]: [c_last_name#52, c_first_name#51, sum#54, isEmpty#55] +Keys [2]: [c_last_name#52, c_first_name#51] +Functions [1]: [sum((cast(ws_quantity#39 as decimal(10,0)) * ws_list_price#40))] + +(85) CometUnion +Child 0 Input [3]: [c_last_name#31, c_first_name#30, sales#56] +Child 1 Input [3]: [c_last_name#52, c_first_name#51, sales#57] + +(86) CometTakeOrderedAndProject +Input [3]: [c_last_name#31, c_first_name#30, sales#56] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#31 ASC NULLS FIRST,c_first_name#30 ASC NULLS FIRST,sales#56 ASC NULLS FIRST], output=[c_last_name#31,c_first_name#30,sales#56]), [c_last_name#31, c_first_name#30, sales#56], 100, [c_last_name#31 ASC NULLS FIRST, c_first_name#30 ASC NULLS FIRST, sales#56 ASC NULLS FIRST], [c_last_name#31, c_first_name#30, sales#56] + +(87) ColumnarToRow [codegen id : 1] +Input [3]: [c_last_name#31, c_first_name#30, sales#56] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (93) -+- * ColumnarToRow (92) - +- CometProject (91) - +- CometFilter (90) - +- CometScan parquet spark_catalog.default.date_dim (89) +BroadcastExchange (92) ++- * ColumnarToRow (91) + +- CometProject (90) + +- CometFilter (89) + +- CometScan parquet spark_catalog.default.date_dim (88) -(89) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#34, d_year#64, d_moy#65] +(88) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#32, d_year#33, d_moy#34] 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 -(90) CometFilter -Input [3]: [d_date_sk#34, d_year#64, d_moy#65] -Condition : ((((isnotnull(d_year#64) AND isnotnull(d_moy#65)) AND (d_year#64 = 2000)) AND (d_moy#65 = 2)) AND isnotnull(d_date_sk#34)) +(89) CometFilter +Input [3]: [d_date_sk#32, d_year#33, d_moy#34] +Condition : ((((isnotnull(d_year#33) AND isnotnull(d_moy#34)) AND (d_year#33 = 2000)) AND (d_moy#34 = 2)) AND isnotnull(d_date_sk#32)) -(91) CometProject -Input [3]: [d_date_sk#34, d_year#64, d_moy#65] -Arguments: [d_date_sk#34], [d_date_sk#34] +(90) CometProject +Input [3]: [d_date_sk#32, d_year#33, d_moy#34] +Arguments: [d_date_sk#32], [d_date_sk#32] -(92) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#34] +(91) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#32] -(93) BroadcastExchange -Input [1]: [d_date_sk#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] +(92) BroadcastExchange +Input [1]: [d_date_sk#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (98) -+- * ColumnarToRow (97) - +- CometProject (96) - +- CometFilter (95) - +- CometScan parquet spark_catalog.default.date_dim (94) +Subquery:2 Hosting operator id = 3 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) -(94) Scan parquet spark_catalog.default.date_dim +(93) 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 -(95) CometFilter +(94) 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)) -(96) CometProject +(95) 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] -(97) ColumnarToRow [codegen id : 1] +(96) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_date#11] -(98) BroadcastExchange +(97) BroadcastExchange Input [2]: [d_date_sk#10, d_date#11] -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#66, ss_quantity#67, ss_sales_price#68, ss_sold_date_sk#69] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] + +Subquery:3 Hosting operator id = 37 Hosting Expression = Subquery scalar-subquery#27, [id=#28] +* ColumnarToRow (115) ++- CometHashAggregate (114) + +- CometColumnarExchange (113) + +- CometHashAggregate (112) + +- CometHashAggregate (111) + +- CometColumnarExchange (110) + +- CometHashAggregate (109) + +- CometProject (108) + +- CometBroadcastHashJoin (107) + :- CometProject (102) + : +- CometBroadcastHashJoin (101) + : :- CometFilter (99) + : : +- CometScan parquet spark_catalog.default.store_sales (98) + : +- ReusedExchange (100) + +- CometBroadcastExchange (106) + +- CometProject (105) + +- CometFilter (104) + +- CometScan parquet spark_catalog.default.date_dim (103) + + +(98) 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#69), dynamicpruningexpression(ss_sold_date_sk#69 IN dynamicpruning#70)] +PartitionFilters: [isnotnull(ss_sold_date_sk#61), dynamicpruningexpression(ss_sold_date_sk#61 IN dynamicpruning#62)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(100) CometFilter -Input [4]: [ss_customer_sk#66, ss_quantity#67, ss_sales_price#68, ss_sold_date_sk#69] -Condition : isnotnull(ss_customer_sk#66) +(99) 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) -(101) ReusedExchange [Reuses operator id: 33] -Output [1]: [c_customer_sk#71] +(100) ReusedExchange [Reuses operator id: 31] +Output [1]: [c_customer_sk#63] -(102) CometBroadcastHashJoin -Left output [4]: [ss_customer_sk#66, ss_quantity#67, ss_sales_price#68, ss_sold_date_sk#69] -Right output [1]: [c_customer_sk#71] -Arguments: [ss_customer_sk#66], [c_customer_sk#71], Inner, BuildRight +(101) 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 -(103) CometProject -Input [5]: [ss_customer_sk#66, ss_quantity#67, ss_sales_price#68, ss_sold_date_sk#69, c_customer_sk#71] -Arguments: [ss_quantity#67, ss_sales_price#68, ss_sold_date_sk#69, c_customer_sk#71], [ss_quantity#67, ss_sales_price#68, ss_sold_date_sk#69, c_customer_sk#71] +(102) 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] -(104) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#72, d_year#73] +(103) 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 -(105) CometFilter -Input [2]: [d_date_sk#72, d_year#73] -Condition : (d_year#73 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#72)) - -(106) CometProject -Input [2]: [d_date_sk#72, d_year#73] -Arguments: [d_date_sk#72], [d_date_sk#72] +(104) 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)) -(107) CometBroadcastExchange -Input [1]: [d_date_sk#72] -Arguments: [d_date_sk#72] +(105) CometProject +Input [2]: [d_date_sk#64, d_year#65] +Arguments: [d_date_sk#64], [d_date_sk#64] -(108) CometBroadcastHashJoin -Left output [4]: [ss_quantity#67, ss_sales_price#68, ss_sold_date_sk#69, c_customer_sk#71] -Right output [1]: [d_date_sk#72] -Arguments: [ss_sold_date_sk#69], [d_date_sk#72], Inner, BuildRight +(106) CometBroadcastExchange +Input [1]: [d_date_sk#64] +Arguments: [d_date_sk#64] -(109) CometProject -Input [5]: [ss_quantity#67, ss_sales_price#68, ss_sold_date_sk#69, c_customer_sk#71, d_date_sk#72] -Arguments: [ss_quantity#67, ss_sales_price#68, c_customer_sk#71], [ss_quantity#67, ss_sales_price#68, c_customer_sk#71] +(107) 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 -(110) CometHashAggregate -Input [3]: [ss_quantity#67, ss_sales_price#68, c_customer_sk#71] -Keys [1]: [c_customer_sk#71] -Functions [1]: [partial_sum((cast(ss_quantity#67 as decimal(10,0)) * ss_sales_price#68))] +(108) 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] -(111) ColumnarToRow [codegen id : 1] -Input [3]: [c_customer_sk#71, sum#74, isEmpty#75] +(109) CometHashAggregate +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))] -(112) Exchange -Input [3]: [c_customer_sk#71, sum#74, isEmpty#75] -Arguments: hashpartitioning(c_customer_sk#71, 5), ENSURE_REQUIREMENTS, [plan_id=12] +(110) CometColumnarExchange +Input [3]: [c_customer_sk#63, sum#66, isEmpty#67] +Arguments: hashpartitioning(c_customer_sk#63, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(113) HashAggregate [codegen id : 2] -Input [3]: [c_customer_sk#71, sum#74, isEmpty#75] -Keys [1]: [c_customer_sk#71] -Functions [1]: [sum((cast(ss_quantity#67 as decimal(10,0)) * ss_sales_price#68))] -Aggregate Attributes [1]: [sum((cast(ss_quantity#67 as decimal(10,0)) * ss_sales_price#68))#76] -Results [1]: [sum((cast(ss_quantity#67 as decimal(10,0)) * ss_sales_price#68))#76 AS csales#77] +(111) CometHashAggregate +Input [3]: [c_customer_sk#63, sum#66, isEmpty#67] +Keys [1]: [c_customer_sk#63] +Functions [1]: [sum((cast(ss_quantity#59 as decimal(10,0)) * ss_sales_price#60))] -(114) HashAggregate [codegen id : 2] -Input [1]: [csales#77] +(112) CometHashAggregate +Input [1]: [csales#68] Keys: [] -Functions [1]: [partial_max(csales#77)] -Aggregate Attributes [1]: [max#78] -Results [1]: [max#79] +Functions [1]: [partial_max(csales#68)] -(115) Exchange -Input [1]: [max#79] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] +(113) CometColumnarExchange +Input [1]: [max#69] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(116) HashAggregate [codegen id : 3] -Input [1]: [max#79] +(114) CometHashAggregate +Input [1]: [max#69] Keys: [] -Functions [1]: [max(csales#77)] -Aggregate Attributes [1]: [max(csales#77)#80] -Results [1]: [max(csales#77)#80 AS tpcds_cmax#81] +Functions [1]: [max(csales#68)] + +(115) ColumnarToRow [codegen id : 1] +Input [1]: [tpcds_cmax#70] -Subquery:4 Hosting operator id = 99 Hosting Expression = ss_sold_date_sk#69 IN dynamicpruning#70 -BroadcastExchange (121) -+- * ColumnarToRow (120) - +- CometProject (119) - +- CometFilter (118) - +- CometScan parquet spark_catalog.default.date_dim (117) +Subquery:4 Hosting operator id = 98 Hosting Expression = ss_sold_date_sk#61 IN dynamicpruning#62 +BroadcastExchange (120) ++- * ColumnarToRow (119) + +- CometProject (118) + +- CometFilter (117) + +- CometScan parquet spark_catalog.default.date_dim (116) -(117) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#72, d_year#73] +(116) 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 -(118) CometFilter -Input [2]: [d_date_sk#72, d_year#73] -Condition : (d_year#73 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#72)) +(117) 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)) -(119) CometProject -Input [2]: [d_date_sk#72, d_year#73] -Arguments: [d_date_sk#72], [d_date_sk#72] +(118) CometProject +Input [2]: [d_date_sk#64, d_year#65] +Arguments: [d_date_sk#64], [d_date_sk#64] -(120) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#72] +(119) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#64] -(121) BroadcastExchange -Input [1]: [d_date_sk#72] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] +(120) BroadcastExchange +Input [1]: [d_date_sk#64] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -Subquery:5 Hosting operator id = 51 Hosting Expression = ReusedSubquery Subquery scalar-subquery#29, [id=#30] +Subquery:5 Hosting operator id = 47 Hosting Expression = ReusedSubquery Subquery scalar-subquery#27, [id=#28] -Subquery:6 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#45 IN dynamicpruning#6 +Subquery:6 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#41 IN dynamicpruning#6 -Subquery:7 Hosting operator id = 74 Hosting Expression = ReusedSubquery Subquery scalar-subquery#29, [id=#30] +Subquery:7 Hosting operator id = 72 Hosting Expression = ReusedSubquery Subquery scalar-subquery#27, [id=#28] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/simplified.txt index 2862f64cc..55f17ca34 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/simplified.txt @@ -1,183 +1,137 @@ -TakeOrderedAndProject [c_last_name,c_first_name,sales] - Union - 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 (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] - Project [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - SortMergeJoin [cs_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (4) - Sort [cs_bill_customer_sk] - InputAdapter - Exchange [cs_bill_customer_sk] #2 - WholeStageCodegen (3) - Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - BroadcastHashJoin [cs_item_sk,item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] - CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] - CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [i_item_sk,i_item_desc] #8 - CometFilter [i_item_sk,i_item_desc] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - InputAdapter - WholeStageCodegen (6) - Sort [c_customer_sk] - Project [c_customer_sk] - Filter [ssales] - Subquery #3 - WholeStageCodegen (3) - HashAggregate [max] [max(csales),tpcds_cmax,max] - InputAdapter - 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] #12 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] - CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] - CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_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 #13 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - 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] #9 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] - CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] - CometProject [ss_customer_sk,ss_quantity,ss_sales_price] - CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [c_customer_sk] #10 - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk] - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (11) - SortMergeJoin [c_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (8) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #16 - WholeStageCodegen (7) +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [c_last_name,c_first_name,sales] + CometUnion [c_last_name,c_first_name,sales] + CometHashAggregate [c_last_name,c_first_name,sales,sum,isEmpty,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price))] + CometColumnarExchange [c_last_name,c_first_name] #1 + CometHashAggregate [c_last_name,c_first_name,sum,isEmpty,cs_quantity,cs_list_price] + CometProject [cs_quantity,cs_list_price,c_first_name,c_last_name] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name,d_date_sk] + CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk,c_first_name,c_last_name] + CometSortMergeJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk] + CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + CometColumnarExchange [cs_bill_customer_sk] #2 + CometProject [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,item_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - InputAdapter - 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] #9 - InputAdapter - ReusedExchange [d_date_sk] #3 - 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] #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] - Project [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - SortMergeJoin [ws_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (18) - Sort [ws_bill_customer_sk] - InputAdapter - 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 - InputAdapter - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [item_sk] #4 - InputAdapter - 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] + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [item_sk] #4 + CometProject [item_sk] + CometFilter [item_sk,cnt] + CometHashAggregate [item_sk,cnt,_groupingexpression,i_item_sk,d_date,count,count(1)] + CometColumnarExchange [_groupingexpression,i_item_sk,d_date] #5 + CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] + CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] + CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] + CometProject [ss_item_sk,d_date] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + CometBroadcastExchange [d_date_sk,d_date] #7 + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + CometBroadcastExchange [i_item_sk,i_item_desc] #8 + CometFilter [i_item_sk,i_item_desc] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + CometSort [c_customer_sk] + CometProject [c_customer_sk] + CometFilter [c_customer_sk,ssales] + Subquery #3 + WholeStageCodegen (1) + ColumnarToRow InputAdapter - ReusedExchange [c_customer_sk,sum,isEmpty] #9 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #15 - InputAdapter - ReusedExchange [d_date_sk] #3 + CometHashAggregate [tpcds_cmax,max,max(csales)] + CometColumnarExchange #11 + CometHashAggregate [max,csales] + CometHashAggregate [csales,c_customer_sk,sum,isEmpty,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + CometColumnarExchange [c_customer_sk] #12 + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] + CometProject [ss_quantity,ss_sales_price,c_customer_sk] + CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] + CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_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 #13 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [c_customer_sk] #10 + CometBroadcastExchange [d_date_sk] #14 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometHashAggregate [c_customer_sk,ssales,sum,isEmpty,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + CometColumnarExchange [c_customer_sk] #9 + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] + CometProject [ss_quantity,ss_sales_price,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] + CometProject [ss_customer_sk,ss_quantity,ss_sales_price] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometBroadcastExchange [c_customer_sk] #10 + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #15 + CometSortMergeJoin [c_customer_sk,c_first_name,c_last_name,c_customer_sk] + CometSort [c_customer_sk,c_first_name,c_last_name] + CometColumnarExchange [c_customer_sk] #16 + CometFilter [c_customer_sk,c_first_name,c_last_name] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] + CometSort [c_customer_sk] + CometProject [c_customer_sk] + CometFilter [c_customer_sk,ssales] + ReusedSubquery [tpcds_cmax] #3 + CometHashAggregate [c_customer_sk,ssales,sum,isEmpty,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + ReusedExchange [c_customer_sk,sum,isEmpty] #9 + CometBroadcastExchange [d_date_sk] #17 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometHashAggregate [c_last_name,c_first_name,sales,sum,isEmpty,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price))] + CometColumnarExchange [c_last_name,c_first_name] #18 + CometHashAggregate [c_last_name,c_first_name,sum,isEmpty,ws_quantity,ws_list_price] + CometProject [ws_quantity,ws_list_price,c_first_name,c_last_name] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name,d_date_sk] + CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk,c_first_name,c_last_name] + CometSortMergeJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk] + CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + CometColumnarExchange [ws_bill_customer_sk] #19 + CometProject [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,item_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [item_sk] #4 + CometSort [c_customer_sk] + CometProject [c_customer_sk] + CometFilter [c_customer_sk,ssales] + ReusedSubquery [tpcds_cmax] #3 + CometHashAggregate [c_customer_sk,ssales,sum,isEmpty,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + ReusedExchange [c_customer_sk,sum,isEmpty] #9 + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #15 + ReusedExchange [d_date_sk] #17 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/explain.txt index 7241b5ea0..7872cef74 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/explain.txt @@ -1,50 +1,50 @@ == Physical Plan == * Filter (46) +- * HashAggregate (45) - +- Exchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- Exchange (41) + +- * ColumnarToRow (44) + +- CometColumnarExchange (43) + +- RowToColumnar (42) + +- * HashAggregate (41) +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Project (14) - : : : : +- * SortMergeJoin Inner (13) - : : : : :- * Sort (6) - : : : : : +- Exchange (5) - : : : : : +- * ColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- * Sort (12) - : : : : +- Exchange (11) - : : : : +- * ColumnarToRow (10) - : : : : +- CometProject (9) - : : : : +- CometFilter (8) - : : : : +- CometScan parquet spark_catalog.default.store_returns (7) - : : : +- BroadcastExchange (19) - : : : +- * ColumnarToRow (18) - : : : +- CometProject (17) - : : : +- CometFilter (16) - : : : +- CometScan parquet spark_catalog.default.store (15) - : : +- BroadcastExchange (25) - : : +- * ColumnarToRow (24) - : : +- CometFilter (23) - : : +- CometScan parquet spark_catalog.default.item (22) - : +- BroadcastExchange (31) - : +- * ColumnarToRow (30) - : +- CometFilter (29) - : +- CometScan parquet spark_catalog.default.customer (28) - +- BroadcastExchange (37) - +- * ColumnarToRow (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.customer_address (34) + +- * ColumnarToRow (39) + +- CometColumnarExchange (38) + +- RowToColumnar (37) + +- * HashAggregate (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * ColumnarToRow (29) + : +- CometProject (28) + : +- CometBroadcastHashJoin (27) + : :- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometProject (18) + : : : +- CometBroadcastHashJoin (17) + : : : :- CometProject (12) + : : : : +- CometSortMergeJoin (11) + : : : : :- CometSort (5) + : : : : : +- CometColumnarExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometSort (10) + : : : : +- CometColumnarExchange (9) + : : : : +- CometProject (8) + : : : : +- CometFilter (7) + : : : : +- CometScan parquet spark_catalog.default.store_returns (6) + : : : +- CometBroadcastExchange (16) + : : : +- CometProject (15) + : : : +- CometFilter (14) + : : : +- CometScan parquet spark_catalog.default.store (13) + : : +- CometBroadcastExchange (21) + : : +- CometFilter (20) + : : +- CometScan parquet spark_catalog.default.item (19) + : +- CometBroadcastExchange (26) + : +- CometFilter (25) + : +- CometScan parquet spark_catalog.default.customer (24) + +- BroadcastExchange (33) + +- * ColumnarToRow (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.customer_address (30) (1) Scan parquet spark_catalog.default.store_sales @@ -62,277 +62,275 @@ Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND is Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -(4) ColumnarToRow [codegen id : 1] +(4) CometColumnarExchange Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(5) Exchange +(5) CometSort Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] -(6) Sort [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST], false, 0 - -(7) Scan parquet spark_catalog.default.store_returns +(6) Scan parquet spark_catalog.default.store_returns Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(8) CometFilter +(7) CometFilter Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) -(9) CometProject +(8) CometProject Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] -(10) ColumnarToRow [codegen id : 3] -Input [2]: [sr_item_sk#7, sr_ticket_number#8] - -(11) Exchange +(9) CometColumnarExchange Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(12) Sort [codegen id : 4] +(10) CometSort Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST], false, 0 +Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] -(13) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_ticket_number#4, ss_item_sk#1] -Right keys [2]: [sr_ticket_number#8, sr_item_sk#7] -Join type: Inner -Join condition: None +(11) CometSortMergeJoin +Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Right output [2]: [sr_item_sk#7, sr_ticket_number#8] +Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner -(14) Project [codegen id : 9] -Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +(12) CometProject Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -(15) Scan parquet spark_catalog.default.store +(13) Scan parquet spark_catalog.default.store Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(16) CometFilter +(14) CometFilter Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(s_zip#14)) -(17) CometProject +(15) CometProject Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Arguments: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14], [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(18) ColumnarToRow [codegen id : 5] +(16) CometBroadcastExchange Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(19) BroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(20) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#10] -Join type: Inner -Join condition: None +(17) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +Right output [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight -(21) Project [codegen id : 9] -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] +(18) CometProject Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] -(22) Scan parquet spark_catalog.default.item +(19) Scan parquet spark_catalog.default.item Output [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale ), IsNotNull(i_item_sk)] ReadSchema: struct -(23) CometFilter +(20) CometFilter Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Condition : ((isnotnull(i_color#18) AND (i_color#18 = pale )) AND isnotnull(i_item_sk#15)) -(24) ColumnarToRow [codegen id : 6] +(21) CometBroadcastExchange Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Arguments: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(25) BroadcastExchange -Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(22) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] +Right output [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Arguments: [ss_item_sk#1], [i_item_sk#15], Inner, BuildRight -(26) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#15] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 9] -Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +(23) CometProject Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(28) Scan parquet spark_catalog.default.customer +(24) Scan parquet spark_catalog.default.customer Output [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct -(29) CometFilter +(25) CometFilter Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_birth_country#24)) -(30) ColumnarToRow [codegen id : 7] -Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] - -(31) BroadcastExchange +(26) CometBroadcastExchange Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +Arguments: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] -(32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#21] -Join type: Inner -Join condition: None +(27) CometBroadcastHashJoin +Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Right output [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] +Arguments: [ss_customer_sk#2], [c_customer_sk#21], Inner, BuildRight -(33) Project [codegen id : 9] -Output [12]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] +(28) CometProject Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] +Arguments: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24], [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] -(34) Scan parquet spark_catalog.default.customer_address +(29) ColumnarToRow [codegen id : 2] +Input [12]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] + +(30) Scan parquet spark_catalog.default.customer_address Output [3]: [ca_state#25, ca_zip#26, ca_country#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct -(35) CometFilter +(31) CometFilter Input [3]: [ca_state#25, ca_zip#26, ca_country#27] Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) -(36) ColumnarToRow [codegen id : 8] +(32) ColumnarToRow [codegen id : 1] Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -(37) BroadcastExchange +(33) BroadcastExchange Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [plan_id=3] -(38) BroadcastHashJoin [codegen id : 9] +(34) BroadcastHashJoin [codegen id : 2] Left keys [2]: [c_birth_country#24, s_zip#14] Right keys [2]: [upper(ca_country#27), ca_zip#26] Join type: Inner Join condition: None -(39) Project [codegen id : 9] +(35) Project [codegen id : 2] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] Input [15]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24, ca_state#25, ca_zip#26, ca_country#27] -(40) HashAggregate [codegen id : 9] +(36) HashAggregate [codegen id : 2] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum#28] Results [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] -(41) Exchange +(37) RowToColumnar +Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] + +(38) CometColumnarExchange +Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] +Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(39) ColumnarToRow [codegen id : 3] Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] -Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(42) HashAggregate [codegen id : 10] +(40) HashAggregate [codegen id : 3] Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#30] Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#30,17,2) AS netpaid#31] -(43) HashAggregate [codegen id : 10] +(41) HashAggregate [codegen id : 3] Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, netpaid#31] Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11] Functions [1]: [partial_sum(netpaid#31)] Aggregate Attributes [2]: [sum#32, isEmpty#33] Results [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] -(44) Exchange +(42) RowToColumnar Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] -Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(45) HashAggregate [codegen id : 11] +(43) CometColumnarExchange +Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] +Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(44) ColumnarToRow [codegen id : 4] +Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] + +(45) HashAggregate [codegen id : 4] Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11] Functions [1]: [sum(netpaid#31)] Aggregate Attributes [1]: [sum(netpaid#31)#36] Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, sum(netpaid#31)#36 AS paid#37] -(46) Filter [codegen id : 11] +(46) Filter [codegen id : 4] Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, paid#37] Condition : (isnotnull(paid#37) AND (cast(paid#37 as decimal(33,8)) > cast(Subquery scalar-subquery#38, [id=#39] as decimal(33,8)))) ===== Subqueries ===== Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#38, [id=#39] -* HashAggregate (73) -+- Exchange (72) - +- * HashAggregate (71) - +- * HashAggregate (70) - +- Exchange (69) - +- * HashAggregate (68) - +- * Project (67) - +- * BroadcastHashJoin Inner BuildRight (66) - :- * Project (64) - : +- * BroadcastHashJoin Inner BuildRight (63) - : :- * Project (61) - : : +- * BroadcastHashJoin Inner BuildRight (60) - : : :- * Project (55) - : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : :- * Project (52) - : : : : +- * SortMergeJoin Inner (51) - : : : : :- * Sort (48) - : : : : : +- ReusedExchange (47) - : : : : +- * Sort (50) - : : : : +- ReusedExchange (49) - : : : +- ReusedExchange (53) - : : +- BroadcastExchange (59) - : : +- * ColumnarToRow (58) - : : +- CometFilter (57) - : : +- CometScan parquet spark_catalog.default.item (56) - : +- ReusedExchange (62) - +- ReusedExchange (65) - - -(47) ReusedExchange [Reuses operator id: 5] +* HashAggregate (77) ++- * ColumnarToRow (76) + +- CometColumnarExchange (75) + +- RowToColumnar (74) + +- * HashAggregate (73) + +- * HashAggregate (72) + +- * ColumnarToRow (71) + +- CometColumnarExchange (70) + +- RowToColumnar (69) + +- * HashAggregate (68) + +- * Project (67) + +- * BroadcastHashJoin Inner BuildRight (66) + :- * ColumnarToRow (64) + : +- CometProject (63) + : +- CometBroadcastHashJoin (62) + : :- CometProject (60) + : : +- CometBroadcastHashJoin (59) + : : :- CometProject (55) + : : : +- CometBroadcastHashJoin (54) + : : : :- CometProject (52) + : : : : +- CometSortMergeJoin (51) + : : : : :- CometSort (48) + : : : : : +- ReusedExchange (47) + : : : : +- CometSort (50) + : : : : +- ReusedExchange (49) + : : : +- ReusedExchange (53) + : : +- CometBroadcastExchange (58) + : : +- CometFilter (57) + : : +- CometScan parquet spark_catalog.default.item (56) + : +- ReusedExchange (61) + +- ReusedExchange (65) + + +(47) ReusedExchange [Reuses operator id: 4] Output [5]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_ticket_number#43, ss_net_paid#44] -(48) Sort [codegen id : 2] +(48) CometSort Input [5]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_ticket_number#43, ss_net_paid#44] -Arguments: [ss_ticket_number#43 ASC NULLS FIRST, ss_item_sk#40 ASC NULLS FIRST], false, 0 +Arguments: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_ticket_number#43, ss_net_paid#44], [ss_ticket_number#43 ASC NULLS FIRST, ss_item_sk#40 ASC NULLS FIRST] -(49) ReusedExchange [Reuses operator id: 11] +(49) ReusedExchange [Reuses operator id: 9] Output [2]: [sr_item_sk#45, sr_ticket_number#46] -(50) Sort [codegen id : 4] +(50) CometSort Input [2]: [sr_item_sk#45, sr_ticket_number#46] -Arguments: [sr_ticket_number#46 ASC NULLS FIRST, sr_item_sk#45 ASC NULLS FIRST], false, 0 +Arguments: [sr_item_sk#45, sr_ticket_number#46], [sr_ticket_number#46 ASC NULLS FIRST, sr_item_sk#45 ASC NULLS FIRST] -(51) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_ticket_number#43, ss_item_sk#40] -Right keys [2]: [sr_ticket_number#46, sr_item_sk#45] -Join type: Inner -Join condition: None +(51) CometSortMergeJoin +Left output [5]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_ticket_number#43, ss_net_paid#44] +Right output [2]: [sr_item_sk#45, sr_ticket_number#46] +Arguments: [ss_ticket_number#43, ss_item_sk#40], [sr_ticket_number#46, sr_item_sk#45], Inner -(52) Project [codegen id : 9] -Output [4]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_net_paid#44] +(52) CometProject Input [7]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_ticket_number#43, ss_net_paid#44, sr_item_sk#45, sr_ticket_number#46] +Arguments: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_net_paid#44], [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_net_paid#44] -(53) ReusedExchange [Reuses operator id: 19] +(53) ReusedExchange [Reuses operator id: 16] Output [4]: [s_store_sk#47, s_store_name#48, s_state#49, s_zip#50] -(54) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#42] -Right keys [1]: [s_store_sk#47] -Join type: Inner -Join condition: None +(54) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_net_paid#44] +Right output [4]: [s_store_sk#47, s_store_name#48, s_state#49, s_zip#50] +Arguments: [ss_store_sk#42], [s_store_sk#47], Inner, BuildRight -(55) Project [codegen id : 9] -Output [6]: [ss_item_sk#40, ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50] +(55) CometProject Input [8]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_net_paid#44, s_store_sk#47, s_store_name#48, s_state#49, s_zip#50] +Arguments: [ss_item_sk#40, ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50], [ss_item_sk#40, ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50] (56) Scan parquet spark_catalog.default.item Output [6]: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] @@ -345,79 +343,89 @@ ReadSchema: struct -(8) CometFilter +(7) CometFilter Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) -(9) CometProject +(8) CometProject Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] -(10) ColumnarToRow [codegen id : 3] -Input [2]: [sr_item_sk#7, sr_ticket_number#8] - -(11) Exchange +(9) CometColumnarExchange Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(12) Sort [codegen id : 4] +(10) CometSort Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST], false, 0 +Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] -(13) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_ticket_number#4, ss_item_sk#1] -Right keys [2]: [sr_ticket_number#8, sr_item_sk#7] -Join type: Inner -Join condition: None +(11) CometSortMergeJoin +Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Right output [2]: [sr_item_sk#7, sr_ticket_number#8] +Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner -(14) Project [codegen id : 9] -Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +(12) CometProject Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -(15) Scan parquet spark_catalog.default.store +(13) Scan parquet spark_catalog.default.store Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(16) CometFilter +(14) CometFilter Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(s_zip#14)) -(17) CometProject +(15) CometProject Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Arguments: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14], [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(18) ColumnarToRow [codegen id : 5] +(16) CometBroadcastExchange Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(19) BroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(20) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#10] -Join type: Inner -Join condition: None +(17) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +Right output [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight -(21) Project [codegen id : 9] -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] +(18) CometProject Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] -(22) Scan parquet spark_catalog.default.item +(19) Scan parquet spark_catalog.default.item Output [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_color), EqualTo(i_color,chiffon ), IsNotNull(i_item_sk)] ReadSchema: struct -(23) CometFilter +(20) CometFilter Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Condition : ((isnotnull(i_color#18) AND (i_color#18 = chiffon )) AND isnotnull(i_item_sk#15)) -(24) ColumnarToRow [codegen id : 6] +(21) CometBroadcastExchange Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Arguments: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(25) BroadcastExchange -Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(22) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] +Right output [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Arguments: [ss_item_sk#1], [i_item_sk#15], Inner, BuildRight -(26) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#15] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 9] -Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +(23) CometProject Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(28) Scan parquet spark_catalog.default.customer +(24) Scan parquet spark_catalog.default.customer Output [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct -(29) CometFilter +(25) CometFilter Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_birth_country#24)) -(30) ColumnarToRow [codegen id : 7] -Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] - -(31) BroadcastExchange +(26) CometBroadcastExchange Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +Arguments: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] -(32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#21] -Join type: Inner -Join condition: None +(27) CometBroadcastHashJoin +Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Right output [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] +Arguments: [ss_customer_sk#2], [c_customer_sk#21], Inner, BuildRight -(33) Project [codegen id : 9] -Output [12]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] +(28) CometProject Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] +Arguments: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24], [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] -(34) Scan parquet spark_catalog.default.customer_address +(29) ColumnarToRow [codegen id : 2] +Input [12]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] + +(30) Scan parquet spark_catalog.default.customer_address Output [3]: [ca_state#25, ca_zip#26, ca_country#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct -(35) CometFilter +(31) CometFilter Input [3]: [ca_state#25, ca_zip#26, ca_country#27] Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) -(36) ColumnarToRow [codegen id : 8] +(32) ColumnarToRow [codegen id : 1] Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -(37) BroadcastExchange +(33) BroadcastExchange Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [plan_id=3] -(38) BroadcastHashJoin [codegen id : 9] +(34) BroadcastHashJoin [codegen id : 2] Left keys [2]: [c_birth_country#24, s_zip#14] Right keys [2]: [upper(ca_country#27), ca_zip#26] Join type: Inner Join condition: None -(39) Project [codegen id : 9] +(35) Project [codegen id : 2] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] Input [15]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24, ca_state#25, ca_zip#26, ca_country#27] -(40) HashAggregate [codegen id : 9] +(36) HashAggregate [codegen id : 2] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum#28] Results [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] -(41) Exchange +(37) RowToColumnar +Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] + +(38) CometColumnarExchange +Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] +Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(39) ColumnarToRow [codegen id : 3] Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] -Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(42) HashAggregate [codegen id : 10] +(40) HashAggregate [codegen id : 3] Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#30] Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#30,17,2) AS netpaid#31] -(43) HashAggregate [codegen id : 10] +(41) HashAggregate [codegen id : 3] Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, netpaid#31] Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11] Functions [1]: [partial_sum(netpaid#31)] Aggregate Attributes [2]: [sum#32, isEmpty#33] Results [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] -(44) Exchange +(42) RowToColumnar Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] -Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(45) HashAggregate [codegen id : 11] +(43) CometColumnarExchange +Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] +Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(44) ColumnarToRow [codegen id : 4] +Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] + +(45) HashAggregate [codegen id : 4] Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11] Functions [1]: [sum(netpaid#31)] Aggregate Attributes [1]: [sum(netpaid#31)#36] Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, sum(netpaid#31)#36 AS paid#37] -(46) Filter [codegen id : 11] +(46) Filter [codegen id : 4] Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, paid#37] Condition : (isnotnull(paid#37) AND (cast(paid#37 as decimal(33,8)) > cast(Subquery scalar-subquery#38, [id=#39] as decimal(33,8)))) ===== Subqueries ===== Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#38, [id=#39] -* HashAggregate (73) -+- Exchange (72) - +- * HashAggregate (71) - +- * HashAggregate (70) - +- Exchange (69) - +- * HashAggregate (68) - +- * Project (67) - +- * BroadcastHashJoin Inner BuildRight (66) - :- * Project (64) - : +- * BroadcastHashJoin Inner BuildRight (63) - : :- * Project (61) - : : +- * BroadcastHashJoin Inner BuildRight (60) - : : :- * Project (55) - : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : :- * Project (52) - : : : : +- * SortMergeJoin Inner (51) - : : : : :- * Sort (48) - : : : : : +- ReusedExchange (47) - : : : : +- * Sort (50) - : : : : +- ReusedExchange (49) - : : : +- ReusedExchange (53) - : : +- BroadcastExchange (59) - : : +- * ColumnarToRow (58) - : : +- CometFilter (57) - : : +- CometScan parquet spark_catalog.default.item (56) - : +- ReusedExchange (62) - +- ReusedExchange (65) - - -(47) ReusedExchange [Reuses operator id: 5] +* HashAggregate (77) ++- * ColumnarToRow (76) + +- CometColumnarExchange (75) + +- RowToColumnar (74) + +- * HashAggregate (73) + +- * HashAggregate (72) + +- * ColumnarToRow (71) + +- CometColumnarExchange (70) + +- RowToColumnar (69) + +- * HashAggregate (68) + +- * Project (67) + +- * BroadcastHashJoin Inner BuildRight (66) + :- * ColumnarToRow (64) + : +- CometProject (63) + : +- CometBroadcastHashJoin (62) + : :- CometProject (60) + : : +- CometBroadcastHashJoin (59) + : : :- CometProject (55) + : : : +- CometBroadcastHashJoin (54) + : : : :- CometProject (52) + : : : : +- CometSortMergeJoin (51) + : : : : :- CometSort (48) + : : : : : +- ReusedExchange (47) + : : : : +- CometSort (50) + : : : : +- ReusedExchange (49) + : : : +- ReusedExchange (53) + : : +- CometBroadcastExchange (58) + : : +- CometFilter (57) + : : +- CometScan parquet spark_catalog.default.item (56) + : +- ReusedExchange (61) + +- ReusedExchange (65) + + +(47) ReusedExchange [Reuses operator id: 4] Output [5]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_ticket_number#43, ss_net_paid#44] -(48) Sort [codegen id : 2] +(48) CometSort Input [5]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_ticket_number#43, ss_net_paid#44] -Arguments: [ss_ticket_number#43 ASC NULLS FIRST, ss_item_sk#40 ASC NULLS FIRST], false, 0 +Arguments: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_ticket_number#43, ss_net_paid#44], [ss_ticket_number#43 ASC NULLS FIRST, ss_item_sk#40 ASC NULLS FIRST] -(49) ReusedExchange [Reuses operator id: 11] +(49) ReusedExchange [Reuses operator id: 9] Output [2]: [sr_item_sk#45, sr_ticket_number#46] -(50) Sort [codegen id : 4] +(50) CometSort Input [2]: [sr_item_sk#45, sr_ticket_number#46] -Arguments: [sr_ticket_number#46 ASC NULLS FIRST, sr_item_sk#45 ASC NULLS FIRST], false, 0 +Arguments: [sr_item_sk#45, sr_ticket_number#46], [sr_ticket_number#46 ASC NULLS FIRST, sr_item_sk#45 ASC NULLS FIRST] -(51) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_ticket_number#43, ss_item_sk#40] -Right keys [2]: [sr_ticket_number#46, sr_item_sk#45] -Join type: Inner -Join condition: None +(51) CometSortMergeJoin +Left output [5]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_ticket_number#43, ss_net_paid#44] +Right output [2]: [sr_item_sk#45, sr_ticket_number#46] +Arguments: [ss_ticket_number#43, ss_item_sk#40], [sr_ticket_number#46, sr_item_sk#45], Inner -(52) Project [codegen id : 9] -Output [4]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_net_paid#44] +(52) CometProject Input [7]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_ticket_number#43, ss_net_paid#44, sr_item_sk#45, sr_ticket_number#46] +Arguments: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_net_paid#44], [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_net_paid#44] -(53) ReusedExchange [Reuses operator id: 19] +(53) ReusedExchange [Reuses operator id: 16] Output [4]: [s_store_sk#47, s_store_name#48, s_state#49, s_zip#50] -(54) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#42] -Right keys [1]: [s_store_sk#47] -Join type: Inner -Join condition: None +(54) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_net_paid#44] +Right output [4]: [s_store_sk#47, s_store_name#48, s_state#49, s_zip#50] +Arguments: [ss_store_sk#42], [s_store_sk#47], Inner, BuildRight -(55) Project [codegen id : 9] -Output [6]: [ss_item_sk#40, ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50] +(55) CometProject Input [8]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_net_paid#44, s_store_sk#47, s_store_name#48, s_state#49, s_zip#50] +Arguments: [ss_item_sk#40, ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50], [ss_item_sk#40, ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50] (56) Scan parquet spark_catalog.default.item Output [6]: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] @@ -345,79 +343,89 @@ ReadSchema: struct -(12) CometFilter +(14) CometFilter Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] Condition : (((isnotnull(ss_quantity#16) AND (ss_quantity#16 >= 6)) AND (ss_quantity#16 <= 10)) AND ((((ss_list_price#18 >= 90.00) AND (ss_list_price#18 <= 100.00)) OR ((ss_coupon_amt#19 >= 2323.00) AND (ss_coupon_amt#19 <= 3323.00))) OR ((ss_wholesale_cost#17 >= 31.00) AND (ss_wholesale_cost#17 <= 51.00)))) -(13) CometProject +(15) CometProject Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] Arguments: [ss_list_price#18], [ss_list_price#18] -(14) CometHashAggregate +(16) CometHashAggregate Input [1]: [ss_list_price#18] Keys [1]: [ss_list_price#18] Functions [2]: [partial_avg(UnscaledValue(ss_list_price#18)), partial_count(ss_list_price#18)] -(15) ColumnarToRow [codegen id : 3] +(17) CometColumnarExchange Input [4]: [ss_list_price#18, sum#21, count#22, count#23] +Arguments: hashpartitioning(ss_list_price#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(16) Exchange +(18) ColumnarToRow [codegen id : 2] Input [4]: [ss_list_price#18, sum#21, count#22, count#23] -Arguments: hashpartitioning(ss_list_price#18, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(17) HashAggregate [codegen id : 4] +(19) HashAggregate [codegen id : 2] Input [4]: [ss_list_price#18, sum#21, count#22, count#23] Keys [1]: [ss_list_price#18] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#18))#24, count(ss_list_price#18)#25] Results [4]: [ss_list_price#18, sum#21, count#22, count#23] -(18) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 2] Input [4]: [ss_list_price#18, sum#21, count#22, count#23] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18), partial_count(distinct ss_list_price#18)] Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#18))#24, count(ss_list_price#18)#25, count(ss_list_price#18)#26] Results [4]: [sum#21, count#22, count#23, count#27] -(19) Exchange +(21) RowToColumnar Input [4]: [sum#21, count#22, count#23, count#27] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] -(20) HashAggregate [codegen id : 5] +(22) CometColumnarExchange +Input [4]: [sum#21, count#22, count#23, count#27] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(23) CometHashAggregate Input [4]: [sum#21, count#22, count#23, count#27] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#18)), count(ss_list_price#18), count(distinct ss_list_price#18)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#18))#24, count(ss_list_price#18)#25, count(ss_list_price#18)#26] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#18))#24 / 100.0) as decimal(11,6)) AS B2_LP#28, count(ss_list_price#18)#25 AS B2_CNT#29, count(ss_list_price#18)#26 AS B2_CNTD#30] -(21) BroadcastExchange +(24) ColumnarToRow [codegen id : 3] +Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] + +(25) BroadcastExchange Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] Arguments: IdentityBroadcastMode, [plan_id=5] -(22) BroadcastNestedLoopJoin [codegen id : 18] +(26) BroadcastNestedLoopJoin [codegen id : 12] Join type: Inner Join condition: None -(23) Scan parquet spark_catalog.default.store_sales +(27) Scan parquet spark_catalog.default.store_sales Output [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,11), LessThanOrEqual(ss_quantity,15), Or(Or(And(GreaterThanOrEqual(ss_list_price,142.00),LessThanOrEqual(ss_list_price,152.00)),And(GreaterThanOrEqual(ss_coupon_amt,12214.00),LessThanOrEqual(ss_coupon_amt,13214.00))),And(GreaterThanOrEqual(ss_wholesale_cost,79.00),LessThanOrEqual(ss_wholesale_cost,99.00)))] ReadSchema: struct -(24) CometFilter +(28) CometFilter Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] Condition : (((isnotnull(ss_quantity#31) AND (ss_quantity#31 >= 11)) AND (ss_quantity#31 <= 15)) AND ((((ss_list_price#33 >= 142.00) AND (ss_list_price#33 <= 152.00)) OR ((ss_coupon_amt#34 >= 12214.00) AND (ss_coupon_amt#34 <= 13214.00))) OR ((ss_wholesale_cost#32 >= 79.00) AND (ss_wholesale_cost#32 <= 99.00)))) -(25) CometProject +(29) CometProject Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] Arguments: [ss_list_price#33], [ss_list_price#33] -(26) CometHashAggregate +(30) CometHashAggregate Input [1]: [ss_list_price#33] Keys [1]: [ss_list_price#33] Functions [2]: [partial_avg(UnscaledValue(ss_list_price#33)), partial_count(ss_list_price#33)] -(27) ColumnarToRow [codegen id : 6] +(31) CometColumnarExchange Input [4]: [ss_list_price#33, sum#36, count#37, count#38] +Arguments: hashpartitioning(ss_list_price#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(28) Exchange +(32) ColumnarToRow [codegen id : 4] Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -Arguments: hashpartitioning(ss_list_price#33, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(29) HashAggregate [codegen id : 7] +(33) HashAggregate [codegen id : 4] Input [4]: [ss_list_price#33, sum#36, count#37, count#38] Keys [1]: [ss_list_price#33] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40] Results [4]: [ss_list_price#33, sum#36, count#37, count#38] -(30) HashAggregate [codegen id : 7] +(34) HashAggregate [codegen id : 4] Input [4]: [ss_list_price#33, sum#36, count#37, count#38] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33), partial_count(distinct ss_list_price#33)] Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40, count(ss_list_price#33)#41] Results [4]: [sum#36, count#37, count#38, count#42] -(31) Exchange +(35) RowToColumnar +Input [4]: [sum#36, count#37, count#38, count#42] + +(36) CometColumnarExchange Input [4]: [sum#36, count#37, count#38, count#42] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(32) HashAggregate [codegen id : 8] +(37) CometHashAggregate Input [4]: [sum#36, count#37, count#38, count#42] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#33)), count(ss_list_price#33), count(distinct ss_list_price#33)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40, count(ss_list_price#33)#41] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#33))#39 / 100.0) as decimal(11,6)) AS B3_LP#43, count(ss_list_price#33)#40 AS B3_CNT#44, count(ss_list_price#33)#41 AS B3_CNTD#45] -(33) BroadcastExchange +(38) ColumnarToRow [codegen id : 5] +Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] + +(39) BroadcastExchange Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] Arguments: IdentityBroadcastMode, [plan_id=8] -(34) BroadcastNestedLoopJoin [codegen id : 18] +(40) BroadcastNestedLoopJoin [codegen id : 12] Join type: Inner Join condition: None -(35) Scan parquet spark_catalog.default.store_sales +(41) Scan parquet spark_catalog.default.store_sales Output [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,16), LessThanOrEqual(ss_quantity,20), Or(Or(And(GreaterThanOrEqual(ss_list_price,135.00),LessThanOrEqual(ss_list_price,145.00)),And(GreaterThanOrEqual(ss_coupon_amt,6071.00),LessThanOrEqual(ss_coupon_amt,7071.00))),And(GreaterThanOrEqual(ss_wholesale_cost,38.00),LessThanOrEqual(ss_wholesale_cost,58.00)))] ReadSchema: struct -(36) CometFilter +(42) CometFilter Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] Condition : (((isnotnull(ss_quantity#46) AND (ss_quantity#46 >= 16)) AND (ss_quantity#46 <= 20)) AND ((((ss_list_price#48 >= 135.00) AND (ss_list_price#48 <= 145.00)) OR ((ss_coupon_amt#49 >= 6071.00) AND (ss_coupon_amt#49 <= 7071.00))) OR ((ss_wholesale_cost#47 >= 38.00) AND (ss_wholesale_cost#47 <= 58.00)))) -(37) CometProject +(43) CometProject Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] Arguments: [ss_list_price#48], [ss_list_price#48] -(38) CometHashAggregate +(44) CometHashAggregate Input [1]: [ss_list_price#48] Keys [1]: [ss_list_price#48] Functions [2]: [partial_avg(UnscaledValue(ss_list_price#48)), partial_count(ss_list_price#48)] -(39) ColumnarToRow [codegen id : 9] +(45) CometColumnarExchange Input [4]: [ss_list_price#48, sum#51, count#52, count#53] +Arguments: hashpartitioning(ss_list_price#48, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(40) Exchange +(46) ColumnarToRow [codegen id : 6] Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -Arguments: hashpartitioning(ss_list_price#48, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(41) HashAggregate [codegen id : 10] +(47) HashAggregate [codegen id : 6] Input [4]: [ss_list_price#48, sum#51, count#52, count#53] Keys [1]: [ss_list_price#48] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55] Results [4]: [ss_list_price#48, sum#51, count#52, count#53] -(42) HashAggregate [codegen id : 10] +(48) HashAggregate [codegen id : 6] Input [4]: [ss_list_price#48, sum#51, count#52, count#53] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48), partial_count(distinct ss_list_price#48)] Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55, count(ss_list_price#48)#56] Results [4]: [sum#51, count#52, count#53, count#57] -(43) Exchange +(49) RowToColumnar Input [4]: [sum#51, count#52, count#53, count#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] -(44) HashAggregate [codegen id : 11] +(50) CometColumnarExchange +Input [4]: [sum#51, count#52, count#53, count#57] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(51) CometHashAggregate Input [4]: [sum#51, count#52, count#53, count#57] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#48)), count(ss_list_price#48), count(distinct ss_list_price#48)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55, count(ss_list_price#48)#56] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#48))#54 / 100.0) as decimal(11,6)) AS B4_LP#58, count(ss_list_price#48)#55 AS B4_CNT#59, count(ss_list_price#48)#56 AS B4_CNTD#60] -(45) BroadcastExchange +(52) ColumnarToRow [codegen id : 7] +Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] + +(53) BroadcastExchange Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] Arguments: IdentityBroadcastMode, [plan_id=11] -(46) BroadcastNestedLoopJoin [codegen id : 18] +(54) BroadcastNestedLoopJoin [codegen id : 12] Join type: Inner Join condition: None -(47) Scan parquet spark_catalog.default.store_sales +(55) Scan parquet spark_catalog.default.store_sales Output [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,25), Or(Or(And(GreaterThanOrEqual(ss_list_price,122.00),LessThanOrEqual(ss_list_price,132.00)),And(GreaterThanOrEqual(ss_coupon_amt,836.00),LessThanOrEqual(ss_coupon_amt,1836.00))),And(GreaterThanOrEqual(ss_wholesale_cost,17.00),LessThanOrEqual(ss_wholesale_cost,37.00)))] ReadSchema: struct -(48) CometFilter +(56) CometFilter Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] Condition : (((isnotnull(ss_quantity#61) AND (ss_quantity#61 >= 21)) AND (ss_quantity#61 <= 25)) AND ((((ss_list_price#63 >= 122.00) AND (ss_list_price#63 <= 132.00)) OR ((ss_coupon_amt#64 >= 836.00) AND (ss_coupon_amt#64 <= 1836.00))) OR ((ss_wholesale_cost#62 >= 17.00) AND (ss_wholesale_cost#62 <= 37.00)))) -(49) CometProject +(57) CometProject Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] Arguments: [ss_list_price#63], [ss_list_price#63] -(50) CometHashAggregate +(58) CometHashAggregate Input [1]: [ss_list_price#63] Keys [1]: [ss_list_price#63] Functions [2]: [partial_avg(UnscaledValue(ss_list_price#63)), partial_count(ss_list_price#63)] -(51) ColumnarToRow [codegen id : 12] +(59) CometColumnarExchange Input [4]: [ss_list_price#63, sum#66, count#67, count#68] +Arguments: hashpartitioning(ss_list_price#63, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(52) Exchange +(60) ColumnarToRow [codegen id : 8] Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -Arguments: hashpartitioning(ss_list_price#63, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(53) HashAggregate [codegen id : 13] +(61) HashAggregate [codegen id : 8] Input [4]: [ss_list_price#63, sum#66, count#67, count#68] Keys [1]: [ss_list_price#63] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70] Results [4]: [ss_list_price#63, sum#66, count#67, count#68] -(54) HashAggregate [codegen id : 13] +(62) HashAggregate [codegen id : 8] Input [4]: [ss_list_price#63, sum#66, count#67, count#68] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63), partial_count(distinct ss_list_price#63)] Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70, count(ss_list_price#63)#71] Results [4]: [sum#66, count#67, count#68, count#72] -(55) Exchange +(63) RowToColumnar Input [4]: [sum#66, count#67, count#68, count#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] -(56) HashAggregate [codegen id : 14] +(64) CometColumnarExchange +Input [4]: [sum#66, count#67, count#68, count#72] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] + +(65) CometHashAggregate Input [4]: [sum#66, count#67, count#68, count#72] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#63)), count(ss_list_price#63), count(distinct ss_list_price#63)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70, count(ss_list_price#63)#71] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#63))#69 / 100.0) as decimal(11,6)) AS B5_LP#73, count(ss_list_price#63)#70 AS B5_CNT#74, count(ss_list_price#63)#71 AS B5_CNTD#75] -(57) BroadcastExchange +(66) ColumnarToRow [codegen id : 9] +Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] + +(67) BroadcastExchange Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] Arguments: IdentityBroadcastMode, [plan_id=14] -(58) BroadcastNestedLoopJoin [codegen id : 18] +(68) BroadcastNestedLoopJoin [codegen id : 12] Join type: Inner Join condition: None -(59) Scan parquet spark_catalog.default.store_sales +(69) Scan parquet spark_catalog.default.store_sales Output [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,26), LessThanOrEqual(ss_quantity,30), Or(Or(And(GreaterThanOrEqual(ss_list_price,154.00),LessThanOrEqual(ss_list_price,164.00)),And(GreaterThanOrEqual(ss_coupon_amt,7326.00),LessThanOrEqual(ss_coupon_amt,8326.00))),And(GreaterThanOrEqual(ss_wholesale_cost,7.00),LessThanOrEqual(ss_wholesale_cost,27.00)))] ReadSchema: struct -(60) CometFilter +(70) CometFilter Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] Condition : (((isnotnull(ss_quantity#76) AND (ss_quantity#76 >= 26)) AND (ss_quantity#76 <= 30)) AND ((((ss_list_price#78 >= 154.00) AND (ss_list_price#78 <= 164.00)) OR ((ss_coupon_amt#79 >= 7326.00) AND (ss_coupon_amt#79 <= 8326.00))) OR ((ss_wholesale_cost#77 >= 7.00) AND (ss_wholesale_cost#77 <= 27.00)))) -(61) CometProject +(71) CometProject Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] Arguments: [ss_list_price#78], [ss_list_price#78] -(62) CometHashAggregate +(72) CometHashAggregate Input [1]: [ss_list_price#78] Keys [1]: [ss_list_price#78] Functions [2]: [partial_avg(UnscaledValue(ss_list_price#78)), partial_count(ss_list_price#78)] -(63) ColumnarToRow [codegen id : 15] +(73) CometColumnarExchange Input [4]: [ss_list_price#78, sum#81, count#82, count#83] +Arguments: hashpartitioning(ss_list_price#78, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] -(64) Exchange +(74) ColumnarToRow [codegen id : 10] Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -Arguments: hashpartitioning(ss_list_price#78, 5), ENSURE_REQUIREMENTS, [plan_id=15] -(65) HashAggregate [codegen id : 16] +(75) HashAggregate [codegen id : 10] Input [4]: [ss_list_price#78, sum#81, count#82, count#83] Keys [1]: [ss_list_price#78] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85] Results [4]: [ss_list_price#78, sum#81, count#82, count#83] -(66) HashAggregate [codegen id : 16] +(76) HashAggregate [codegen id : 10] Input [4]: [ss_list_price#78, sum#81, count#82, count#83] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78), partial_count(distinct ss_list_price#78)] Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85, count(ss_list_price#78)#86] Results [4]: [sum#81, count#82, count#83, count#87] -(67) Exchange +(77) RowToColumnar +Input [4]: [sum#81, count#82, count#83, count#87] + +(78) CometColumnarExchange Input [4]: [sum#81, count#82, count#83, count#87] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=16] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] -(68) HashAggregate [codegen id : 17] +(79) CometHashAggregate Input [4]: [sum#81, count#82, count#83, count#87] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#78)), count(ss_list_price#78), count(distinct ss_list_price#78)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85, count(ss_list_price#78)#86] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#78))#84 / 100.0) as decimal(11,6)) AS B6_LP#88, count(ss_list_price#78)#85 AS B6_CNT#89, count(ss_list_price#78)#86 AS B6_CNTD#90] -(69) BroadcastExchange +(80) ColumnarToRow [codegen id : 11] +Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] + +(81) BroadcastExchange Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] Arguments: IdentityBroadcastMode, [plan_id=17] -(70) BroadcastNestedLoopJoin [codegen id : 18] +(82) BroadcastNestedLoopJoin [codegen id : 12] Join type: Inner Join condition: None diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/simplified.txt index 80c886708..73343f39e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/simplified.txt @@ -1,110 +1,110 @@ -WholeStageCodegen (18) +WholeStageCodegen (12) BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B1_LP,B1_CNT,B1_CNTD,sum,count,count,count] + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (2) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + CometHashAggregate [B1_LP,B1_CNT,B1_CNTD,sum,count,count,count,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] + CometColumnarExchange #1 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + ColumnarToRow + InputAdapter + CometColumnarExchange [ss_list_price] #2 CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #3 - WholeStageCodegen (5) - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B2_LP,B2_CNT,B2_CNTD,sum,count,count,count] + WholeStageCodegen (3) + ColumnarToRow InputAdapter - Exchange #4 - WholeStageCodegen (4) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter + CometHashAggregate [B2_LP,B2_CNT,B2_CNTD,sum,count,count,count,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] + CometColumnarExchange #4 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + ColumnarToRow + InputAdapter + CometColumnarExchange [ss_list_price] #5 CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #6 - WholeStageCodegen (8) - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B3_LP,B3_CNT,B3_CNTD,sum,count,count,count] + WholeStageCodegen (5) + ColumnarToRow InputAdapter - Exchange #7 - WholeStageCodegen (7) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #8 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter + CometHashAggregate [B3_LP,B3_CNT,B3_CNTD,sum,count,count,count,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] + CometColumnarExchange #7 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + ColumnarToRow + InputAdapter + CometColumnarExchange [ss_list_price] #8 CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #9 - WholeStageCodegen (11) - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B4_LP,B4_CNT,B4_CNTD,sum,count,count,count] + WholeStageCodegen (7) + ColumnarToRow InputAdapter - Exchange #10 - WholeStageCodegen (10) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #11 - WholeStageCodegen (9) - ColumnarToRow - InputAdapter + CometHashAggregate [B4_LP,B4_CNT,B4_CNTD,sum,count,count,count,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] + CometColumnarExchange #10 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + ColumnarToRow + InputAdapter + CometColumnarExchange [ss_list_price] #11 CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #12 - WholeStageCodegen (14) - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B5_LP,B5_CNT,B5_CNTD,sum,count,count,count] + WholeStageCodegen (9) + ColumnarToRow InputAdapter - Exchange #13 - WholeStageCodegen (13) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #14 - WholeStageCodegen (12) - ColumnarToRow - InputAdapter + CometHashAggregate [B5_LP,B5_CNT,B5_CNTD,sum,count,count,count,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] + CometColumnarExchange #13 + RowToColumnar + WholeStageCodegen (8) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + ColumnarToRow + InputAdapter + CometColumnarExchange [ss_list_price] #14 CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #15 - WholeStageCodegen (17) - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B6_LP,B6_CNT,B6_CNTD,sum,count,count,count] + WholeStageCodegen (11) + ColumnarToRow InputAdapter - Exchange #16 - WholeStageCodegen (16) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #17 - WholeStageCodegen (15) - ColumnarToRow - InputAdapter + CometHashAggregate [B6_LP,B6_CNT,B6_CNTD,sum,count,count,count,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] + CometColumnarExchange #16 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + ColumnarToRow + InputAdapter + CometColumnarExchange [ss_list_price] #17 CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/explain.txt index 3c350ad07..a3346549e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (45) -+- * HashAggregate (44) - +- Exchange (43) - +- * ColumnarToRow (42) +* ColumnarToRow (45) ++- CometTakeOrderedAndProject (44) + +- CometHashAggregate (43) + +- CometColumnarExchange (42) +- CometHashAggregate (41) +- CometProject (40) +- CometBroadcastHashJoin (39) @@ -245,23 +245,21 @@ Input [7]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#28, 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)] -(42) ColumnarToRow [codegen id : 1] +(42) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(43) CometHashAggregate 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)#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] -(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] +(44) CometTakeOrderedAndProject +Input [7]: [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29, store_sales_quantity#36, store_returns_quantity#37, catalog_sales_quantity#38] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[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], output=[i_item_id#31,i_item_desc#32,s_store_id#28,s_store_name#29,store_sales_quantity#36,store_returns_quantity#37,catalog_sales_quantity#38]), [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29, store_sales_quantity#36, store_returns_quantity#37, catalog_sales_quantity#38], 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#36, store_returns_quantity#37, catalog_sales_quantity#38] + +(45) ColumnarToRow [codegen id : 1] +Input [7]: [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29, store_sales_quantity#36, store_returns_quantity#37, catalog_sales_quantity#38] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/simplified.txt index 88b4ecd1d..3ca13e7b4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/simplified.txt @@ -1,73 +1,71 @@ -TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum,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,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc] - CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_store_id,s_store_name] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,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 [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_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_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_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 [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,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 - CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - 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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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 + CometTakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity] + CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity,sum,sum,sum,sum(ss_quantity),sum(sr_return_quantity),sum(cs_quantity)] + CometColumnarExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 + CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum,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,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc] + CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_store_id,s_store_name] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,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 [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_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_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_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 [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,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + 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_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #8 + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #9 + CometBroadcastExchange [d_date_sk] #8 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #10 - CometFilter [s_store_sk,s_store_id,s_store_name] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #11 - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk] #9 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #10 + CometFilter [s_store_sk,s_store_id,s_store_name] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #11 + CometFilter [i_item_sk,i_item_id,i_item_desc] + 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-spark3_5/q3/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/explain.txt index 98092ef29..9079e0886 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (19) -+- * HashAggregate (18) - +- Exchange (17) - +- * ColumnarToRow (16) +* ColumnarToRow (19) ++- CometTakeOrderedAndProject (18) + +- CometHashAggregate (17) + +- CometColumnarExchange (16) +- CometHashAggregate (15) +- CometProject (14) +- CometBroadcastHashJoin (13) @@ -93,21 +93,19 @@ 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))] -(16) ColumnarToRow [codegen id : 1] +(16) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(18) HashAggregate [codegen id : 2] +(17) CometHashAggregate 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))#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] -(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] +(18) CometTakeOrderedAndProject +Input [4]: [d_year#2, brand_id#12, brand#13, sum_agg#14] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,sum_agg#14 DESC NULLS LAST,brand_id#12 ASC NULLS FIRST], output=[d_year#2,brand_id#12,brand#13,sum_agg#14]), [d_year#2, brand_id#12, brand#13, sum_agg#14], 100, [d_year#2 ASC NULLS FIRST, sum_agg#14 DESC NULLS LAST, brand_id#12 ASC NULLS FIRST], [d_year#2, brand_id#12, brand#13, sum_agg#14] + +(19) ColumnarToRow [codegen id : 1] +Input [4]: [d_year#2, brand_id#12, brand#13, sum_agg#14] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/simplified.txt index bccda58a4..65c80baed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/simplified.txt @@ -1,23 +1,21 @@ -TakeOrderedAndProject [d_year,sum_agg,brand_id,brand] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [d_year,i_brand,i_brand_id,sum,ss_ext_sales_price] - CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] - CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [d_year,brand_id,brand,sum_agg] + CometHashAggregate [d_year,brand_id,brand,sum_agg,i_brand,i_brand_id,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange [d_year,i_brand,i_brand_id] #1 + CometHashAggregate [d_year,i_brand,i_brand_id,sum,ss_ext_sales_price] + CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] + CometProject [d_year,ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometProject [d_date_sk,d_year] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 + CometProject [i_item_sk,i_brand_id,i_brand] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id] + 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-spark3_5/q30/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/explain.txt index f4a0a6d51..a14fa6074 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/explain.txt @@ -1,55 +1,52 @@ == Physical Plan == -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) +* ColumnarToRow (48) ++- CometTakeOrderedAndProject (47) + +- CometProject (46) + +- CometBroadcastHashJoin (45) + :- CometProject (40) + : +- CometBroadcastHashJoin (39) + : :- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometFilter (17) + : : : +- CometHashAggregate (16) + : : : +- CometColumnarExchange (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) + : : +- CometBroadcastExchange (33) + : : +- CometFilter (32) + : : +- CometHashAggregate (31) + : : +- CometColumnarExchange (30) + : : +- CometHashAggregate (29) + : : +- CometHashAggregate (28) + : : +- CometColumnarExchange (27) + : : +- CometHashAggregate (26) + : : +- CometProject (25) + : : +- CometBroadcastHashJoin (24) + : : :- CometProject (22) + : : : +- CometBroadcastHashJoin (21) + : : : :- CometFilter (19) + : : : : +- CometScan parquet spark_catalog.default.web_returns (18) + : : : +- ReusedExchange (20) + : : +- ReusedExchange (23) + : +- CometBroadcastExchange (38) + : +- CometFilter (37) + : +- CometScan parquet spark_catalog.default.customer (36) + +- CometBroadcastExchange (44) + +- CometProject (43) + +- CometFilter (42) + +- CometScan parquet spark_catalog.default.customer_address (41) (1) Scan parquet spark_catalog.default.web_returns @@ -121,211 +118,191 @@ 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))] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(17) HashAggregate [codegen id : 7] +(16) CometHashAggregate 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#9 AS ctr_state#13, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#11,17,2) AS ctr_total_return#14] -(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) CometFilter +Input [3]: [ctr_customer_sk#11, ctr_state#12, ctr_total_return#13] +Condition : isnotnull(ctr_total_return#13) -(19) 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] +(18) Scan parquet spark_catalog.default.web_returns +Output [4]: [wr_returning_customer_sk#14, wr_returning_addr_sk#15, wr_return_amt#16, wr_returned_date_sk#17] 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#17), dynamicpruningexpression(wr_returned_date_sk#17 IN dynamicpruning#18)] PushedFilters: [IsNotNull(wr_returning_addr_sk)] ReadSchema: struct -(20) 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) - -(21) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#20] - -(22) CometBroadcastHashJoin -Left output [4]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18] -Right output [1]: [d_date_sk#20] -Arguments: [wr_returned_date_sk#18], [d_date_sk#20], Inner, BuildRight - -(23) CometProject -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] -Arguments: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17], [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17] - -(24) ReusedExchange [Reuses operator id: 11] -Output [2]: [ca_address_sk#21, ca_state#22] - -(25) CometBroadcastHashJoin -Left output [3]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17] -Right output [2]: [ca_address_sk#21, ca_state#22] -Arguments: [wr_returning_addr_sk#16], [ca_address_sk#21], Inner, BuildRight - -(26) CometProject -Input [5]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, ca_address_sk#21, ca_state#22] -Arguments: [wr_returning_customer_sk#15, wr_return_amt#17, ca_state#22], [wr_returning_customer_sk#15, wr_return_amt#17, ca_state#22] - -(27) CometHashAggregate -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))] - -(28) ColumnarToRow [codegen id : 2] -Input [3]: [wr_returning_customer_sk#15, ca_state#22, sum#23] - -(29) Exchange -Input [3]: [wr_returning_customer_sk#15, ca_state#22, sum#23] -Arguments: hashpartitioning(wr_returning_customer_sk#15, ca_state#22, 5), ENSURE_REQUIREMENTS, [plan_id=2] - -(30) HashAggregate [codegen id : 3] -Input [3]: [wr_returning_customer_sk#15, ca_state#22, sum#23] -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#24, MakeDecimal(sum(UnscaledValue(wr_return_amt#17))#11,17,2) AS ctr_total_return#25] - -(31) HashAggregate [codegen id : 3] -Input [2]: [ctr_state#24, ctr_total_return#25] -Keys [1]: [ctr_state#24] -Functions [1]: [partial_avg(ctr_total_return#25)] -Aggregate Attributes [2]: [sum#26, count#27] -Results [3]: [ctr_state#24, sum#28, count#29] - -(32) Exchange -Input [3]: [ctr_state#24, sum#28, count#29] -Arguments: hashpartitioning(ctr_state#24, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(33) HashAggregate [codegen id : 4] -Input [3]: [ctr_state#24, sum#28, count#29] -Keys [1]: [ctr_state#24] -Functions [1]: [avg(ctr_total_return#25)] -Aggregate Attributes [1]: [avg(ctr_total_return#25)#30] -Results [2]: [(avg(ctr_total_return#25)#30 * 1.2) AS (avg(ctr_total_return) * 1.2)#31, ctr_state#24] - -(34) Filter [codegen id : 4] -Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_state#24] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#31) - -(35) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_state#24] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=4] - -(36) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ctr_state#13] -Right keys [1]: [ctr_state#24] -Join type: Inner -Join condition: (cast(ctr_total_return#14 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#31) - -(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)#31, ctr_state#24] - -(38) Scan parquet spark_catalog.default.customer -Output [14]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] +(19) CometFilter +Input [4]: [wr_returning_customer_sk#14, wr_returning_addr_sk#15, wr_return_amt#16, wr_returned_date_sk#17] +Condition : isnotnull(wr_returning_addr_sk#15) + +(20) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#19] + +(21) CometBroadcastHashJoin +Left output [4]: [wr_returning_customer_sk#14, wr_returning_addr_sk#15, wr_return_amt#16, wr_returned_date_sk#17] +Right output [1]: [d_date_sk#19] +Arguments: [wr_returned_date_sk#17], [d_date_sk#19], Inner, BuildRight + +(22) CometProject +Input [5]: [wr_returning_customer_sk#14, wr_returning_addr_sk#15, wr_return_amt#16, wr_returned_date_sk#17, d_date_sk#19] +Arguments: [wr_returning_customer_sk#14, wr_returning_addr_sk#15, wr_return_amt#16], [wr_returning_customer_sk#14, wr_returning_addr_sk#15, wr_return_amt#16] + +(23) ReusedExchange [Reuses operator id: 11] +Output [2]: [ca_address_sk#20, ca_state#21] + +(24) CometBroadcastHashJoin +Left output [3]: [wr_returning_customer_sk#14, wr_returning_addr_sk#15, wr_return_amt#16] +Right output [2]: [ca_address_sk#20, ca_state#21] +Arguments: [wr_returning_addr_sk#15], [ca_address_sk#20], Inner, BuildRight + +(25) CometProject +Input [5]: [wr_returning_customer_sk#14, wr_returning_addr_sk#15, wr_return_amt#16, ca_address_sk#20, ca_state#21] +Arguments: [wr_returning_customer_sk#14, wr_return_amt#16, ca_state#21], [wr_returning_customer_sk#14, wr_return_amt#16, ca_state#21] + +(26) CometHashAggregate +Input [3]: [wr_returning_customer_sk#14, wr_return_amt#16, ca_state#21] +Keys [2]: [wr_returning_customer_sk#14, ca_state#21] +Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#16))] + +(27) CometColumnarExchange +Input [3]: [wr_returning_customer_sk#14, ca_state#21, sum#22] +Arguments: hashpartitioning(wr_returning_customer_sk#14, ca_state#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(28) CometHashAggregate +Input [3]: [wr_returning_customer_sk#14, ca_state#21, sum#22] +Keys [2]: [wr_returning_customer_sk#14, ca_state#21] +Functions [1]: [sum(UnscaledValue(wr_return_amt#16))] + +(29) CometHashAggregate +Input [2]: [ctr_state#23, ctr_total_return#24] +Keys [1]: [ctr_state#23] +Functions [1]: [partial_avg(ctr_total_return#24)] + +(30) CometColumnarExchange +Input [3]: [ctr_state#23, sum#25, count#26] +Arguments: hashpartitioning(ctr_state#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(31) CometHashAggregate +Input [3]: [ctr_state#23, sum#25, count#26] +Keys [1]: [ctr_state#23] +Functions [1]: [avg(ctr_total_return#24)] + +(32) CometFilter +Input [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_state#23] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#27) + +(33) CometBroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_state#23] +Arguments: [(avg(ctr_total_return) * 1.2)#27, ctr_state#23] + +(34) CometBroadcastHashJoin +Left output [3]: [ctr_customer_sk#11, ctr_state#12, ctr_total_return#13] +Right output [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_state#23] +Arguments: [ctr_state#12], [ctr_state#23], Inner, (cast(ctr_total_return#13 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#27), BuildRight + +(35) CometProject +Input [5]: [ctr_customer_sk#11, ctr_state#12, ctr_total_return#13, (avg(ctr_total_return) * 1.2)#27, ctr_state#23] +Arguments: [ctr_customer_sk#11, ctr_total_return#13], [ctr_customer_sk#11, ctr_total_return#13] + +(36) Scan parquet spark_catalog.default.customer +Output [14]: [c_customer_sk#28, c_customer_id#29, c_current_addr_sk#30, c_salutation#31, c_first_name#32, c_last_name#33, c_preferred_cust_flag#34, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#39, c_email_address#40, c_last_review_date#41] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(39) CometFilter -Input [14]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] -Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_current_addr_sk#34)) - -(40) ColumnarToRow [codegen id : 5] -Input [14]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] +(37) CometFilter +Input [14]: [c_customer_sk#28, c_customer_id#29, c_current_addr_sk#30, c_salutation#31, c_first_name#32, c_last_name#33, c_preferred_cust_flag#34, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#39, c_email_address#40, c_last_review_date#41] +Condition : (isnotnull(c_customer_sk#28) AND isnotnull(c_current_addr_sk#30)) -(41) BroadcastExchange -Input [14]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +(38) CometBroadcastExchange +Input [14]: [c_customer_sk#28, c_customer_id#29, c_current_addr_sk#30, c_salutation#31, c_first_name#32, c_last_name#33, c_preferred_cust_flag#34, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#39, c_email_address#40, c_last_review_date#41] +Arguments: [c_customer_sk#28, c_customer_id#29, c_current_addr_sk#30, c_salutation#31, c_first_name#32, c_last_name#33, c_preferred_cust_flag#34, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#39, c_email_address#40, c_last_review_date#41] -(42) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ctr_customer_sk#12] -Right keys [1]: [c_customer_sk#32] -Join type: Inner -Join condition: None +(39) CometBroadcastHashJoin +Left output [2]: [ctr_customer_sk#11, ctr_total_return#13] +Right output [14]: [c_customer_sk#28, c_customer_id#29, c_current_addr_sk#30, c_salutation#31, c_first_name#32, c_last_name#33, c_preferred_cust_flag#34, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#39, c_email_address#40, c_last_review_date#41] +Arguments: [ctr_customer_sk#11], [c_customer_sk#28], Inner, BuildRight -(43) Project [codegen id : 7] -Output [14]: [ctr_total_return#14, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] -Input [16]: [ctr_customer_sk#12, ctr_total_return#14, c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] +(40) CometProject +Input [16]: [ctr_customer_sk#11, ctr_total_return#13, c_customer_sk#28, c_customer_id#29, c_current_addr_sk#30, c_salutation#31, c_first_name#32, c_last_name#33, c_preferred_cust_flag#34, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#39, c_email_address#40, c_last_review_date#41] +Arguments: [ctr_total_return#13, c_customer_id#29, c_current_addr_sk#30, c_salutation#31, c_first_name#32, c_last_name#33, c_preferred_cust_flag#34, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#39, c_email_address#40, c_last_review_date#41], [ctr_total_return#13, c_customer_id#29, c_current_addr_sk#30, c_salutation#31, c_first_name#32, c_last_name#33, c_preferred_cust_flag#34, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#39, c_email_address#40, c_last_review_date#41] -(44) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#46, ca_state#47] +(41) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#42, ca_state#43] 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 -(45) CometFilter -Input [2]: [ca_address_sk#46, ca_state#47] -Condition : ((isnotnull(ca_state#47) AND (ca_state#47 = GA)) AND isnotnull(ca_address_sk#46)) +(42) CometFilter +Input [2]: [ca_address_sk#42, ca_state#43] +Condition : ((isnotnull(ca_state#43) AND (ca_state#43 = GA)) AND isnotnull(ca_address_sk#42)) -(46) CometProject -Input [2]: [ca_address_sk#46, ca_state#47] -Arguments: [ca_address_sk#46], [ca_address_sk#46] +(43) CometProject +Input [2]: [ca_address_sk#42, ca_state#43] +Arguments: [ca_address_sk#42], [ca_address_sk#42] -(47) ColumnarToRow [codegen id : 6] -Input [1]: [ca_address_sk#46] +(44) CometBroadcastExchange +Input [1]: [ca_address_sk#42] +Arguments: [ca_address_sk#42] -(48) BroadcastExchange -Input [1]: [ca_address_sk#46] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +(45) CometBroadcastHashJoin +Left output [14]: [ctr_total_return#13, c_customer_id#29, c_current_addr_sk#30, c_salutation#31, c_first_name#32, c_last_name#33, c_preferred_cust_flag#34, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#39, c_email_address#40, c_last_review_date#41] +Right output [1]: [ca_address_sk#42] +Arguments: [c_current_addr_sk#30], [ca_address_sk#42], Inner, BuildRight -(49) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#34] -Right keys [1]: [ca_address_sk#46] -Join type: Inner -Join condition: None +(46) CometProject +Input [15]: [ctr_total_return#13, c_customer_id#29, c_current_addr_sk#30, c_salutation#31, c_first_name#32, c_last_name#33, c_preferred_cust_flag#34, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#39, c_email_address#40, c_last_review_date#41, ca_address_sk#42] +Arguments: [c_customer_id#29, c_salutation#31, c_first_name#32, c_last_name#33, c_preferred_cust_flag#34, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#39, c_email_address#40, c_last_review_date#41, ctr_total_return#13], [c_customer_id#29, c_salutation#31, c_first_name#32, c_last_name#33, c_preferred_cust_flag#34, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#39, c_email_address#40, c_last_review_date#41, ctr_total_return#13] -(50) Project [codegen id : 7] -Output [13]: [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45, ctr_total_return#14] -Input [15]: [ctr_total_return#14, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45, ca_address_sk#46] +(47) CometTakeOrderedAndProject +Input [13]: [c_customer_id#29, c_salutation#31, c_first_name#32, c_last_name#33, c_preferred_cust_flag#34, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#39, c_email_address#40, c_last_review_date#41, ctr_total_return#13] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#29 ASC NULLS FIRST,c_salutation#31 ASC NULLS FIRST,c_first_name#32 ASC NULLS FIRST,c_last_name#33 ASC NULLS FIRST,c_preferred_cust_flag#34 ASC NULLS FIRST,c_birth_day#35 ASC NULLS FIRST,c_birth_month#36 ASC NULLS FIRST,c_birth_year#37 ASC NULLS FIRST,c_birth_country#38 ASC NULLS FIRST,c_login#39 ASC NULLS FIRST,c_email_address#40 ASC NULLS FIRST,c_last_review_date#41 ASC NULLS FIRST,ctr_total_return#13 ASC NULLS FIRST], output=[c_customer_id#29,c_salutation#31,c_first_name#32,c_last_name#33,c_preferred_cust_flag#34,c_birth_day#35,c_birth_month#36,c_birth_year#37,c_birth_country#38,c_login#39,c_email_address#40,c_last_review_date#41,ctr_total_return#13]), [c_customer_id#29, c_salutation#31, c_first_name#32, c_last_name#33, c_preferred_cust_flag#34, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#39, c_email_address#40, c_last_review_date#41, ctr_total_return#13], 100, [c_customer_id#29 ASC NULLS FIRST, c_salutation#31 ASC NULLS FIRST, c_first_name#32 ASC NULLS FIRST, c_last_name#33 ASC NULLS FIRST, c_preferred_cust_flag#34 ASC NULLS FIRST, c_birth_day#35 ASC NULLS FIRST, c_birth_month#36 ASC NULLS FIRST, c_birth_year#37 ASC NULLS FIRST, c_birth_country#38 ASC NULLS FIRST, c_login#39 ASC NULLS FIRST, c_email_address#40 ASC NULLS FIRST, c_last_review_date#41 ASC NULLS FIRST, ctr_total_return#13 ASC NULLS FIRST], [c_customer_id#29, c_salutation#31, c_first_name#32, c_last_name#33, c_preferred_cust_flag#34, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#39, c_email_address#40, c_last_review_date#41, ctr_total_return#13] -(51) TakeOrderedAndProject -Input [13]: [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45, ctr_total_return#14] -Arguments: 100, [c_customer_id#33 ASC NULLS FIRST, c_salutation#35 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, c_last_name#37 ASC NULLS FIRST, c_preferred_cust_flag#38 ASC NULLS FIRST, c_birth_day#39 ASC NULLS FIRST, c_birth_month#40 ASC NULLS FIRST, c_birth_year#41 ASC NULLS FIRST, c_birth_country#42 ASC NULLS FIRST, c_login#43 ASC NULLS FIRST, c_email_address#44 ASC NULLS FIRST, c_last_review_date#45 ASC NULLS FIRST, ctr_total_return#14 ASC NULLS FIRST], [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45, ctr_total_return#14] +(48) ColumnarToRow [codegen id : 1] +Input [13]: [c_customer_id#29, c_salutation#31, c_first_name#32, c_last_name#33, c_preferred_cust_flag#34, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#39, c_email_address#40, c_last_review_date#41, ctr_total_return#13] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (56) -+- * ColumnarToRow (55) - +- CometProject (54) - +- CometFilter (53) - +- CometScan parquet spark_catalog.default.date_dim (52) +BroadcastExchange (53) ++- * ColumnarToRow (52) + +- CometProject (51) + +- CometFilter (50) + +- CometScan parquet spark_catalog.default.date_dim (49) -(52) Scan parquet spark_catalog.default.date_dim +(49) 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 -(53) CometFilter +(50) 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)) -(54) CometProject +(51) CometProject Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(55) ColumnarToRow [codegen id : 1] +(52) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(56) BroadcastExchange +(53) BroadcastExchange Input [1]: [d_date_sk#6] -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=4] -Subquery:2 Hosting operator id = 19 Hosting Expression = wr_returned_date_sk#18 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 18 Hosting Expression = wr_returned_date_sk#17 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/simplified.txt index a80e0ca71..576a8779f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/simplified.txt @@ -1,77 +1,59 @@ -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 (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] - BroadcastHashJoin [ctr_customer_sk,c_customer_sk] - Project [ctr_customer_sk,ctr_total_return] - BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] - Filter [ctr_total_return] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [wr_returning_customer_sk,ca_state,sum,wr_return_amt] - CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] - CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] - CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [ca_address_sk,ca_state] #4 - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - 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] #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] #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometHashAggregate [wr_returning_customer_sk,ca_state,sum,wr_return_amt] - CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] - CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] - CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_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 - ReusedExchange [d_date_sk] #3 - ReusedExchange [ca_address_sk,ca_state] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometFilter [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] - 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 #9 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [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] + CometProject [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] + CometBroadcastHashJoin [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,ca_address_sk] + CometProject [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] + CometBroadcastHashJoin [ctr_customer_sk,ctr_total_return,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] + CometProject [ctr_customer_sk,ctr_total_return] + CometBroadcastHashJoin [ctr_customer_sk,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_state] + CometFilter [ctr_customer_sk,ctr_state,ctr_total_return] + CometHashAggregate [ctr_customer_sk,ctr_state,ctr_total_return,wr_returning_customer_sk,ca_state,sum,sum(UnscaledValue(wr_return_amt))] + CometColumnarExchange [wr_returning_customer_sk,ca_state] #1 + CometHashAggregate [wr_returning_customer_sk,ca_state,sum,wr_return_amt] + CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] + CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] + CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [ca_address_sk,ca_state] #4 + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_state] #5 + CometFilter [(avg(ctr_total_return) * 1.2),ctr_state] + CometHashAggregate [(avg(ctr_total_return) * 1.2),ctr_state,sum,count,avg(ctr_total_return)] + CometColumnarExchange [ctr_state] #6 + CometHashAggregate [ctr_state,sum,count,ctr_total_return] + CometHashAggregate [ctr_state,ctr_total_return,wr_returning_customer_sk,ca_state,sum,sum(UnscaledValue(wr_return_amt))] + CometColumnarExchange [wr_returning_customer_sk,ca_state] #7 + CometHashAggregate [wr_returning_customer_sk,ca_state,sum,wr_return_amt] + CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] + CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] + CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_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 + ReusedExchange [d_date_sk] #3 + ReusedExchange [ca_address_sk,ca_state] #4 + CometBroadcastExchange [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] #8 + CometFilter [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] + 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] + CometBroadcastExchange [ca_address_sk] #9 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/explain.txt index d63ab7356..7284a48b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/explain.txt @@ -1,99 +1,94 @@ == Physical Plan == -* 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) +* ColumnarToRow (90) ++- CometSort (89) + +- CometColumnarExchange (88) + +- CometProject (87) + +- CometBroadcastHashJoin (86) + :- CometProject (73) + : +- CometBroadcastHashJoin (72) + : :- CometBroadcastHashJoin (59) + : : :- CometProject (46) + : : : +- CometBroadcastHashJoin (45) + : : : :- CometBroadcastHashJoin (30) + : : : : :- CometHashAggregate (15) + : : : : : +- CometColumnarExchange (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) + : : : : +- CometBroadcastExchange (29) + : : : : +- CometHashAggregate (28) + : : : : +- CometColumnarExchange (27) + : : : : +- CometHashAggregate (26) + : : : : +- CometProject (25) + : : : : +- CometBroadcastHashJoin (24) + : : : : :- CometProject (22) + : : : : : +- CometBroadcastHashJoin (21) + : : : : : :- CometFilter (17) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (16) + : : : : : +- CometBroadcastExchange (20) + : : : : : +- CometFilter (19) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (18) + : : : : +- ReusedExchange (23) + : : : +- CometBroadcastExchange (44) + : : : +- CometHashAggregate (43) + : : : +- CometColumnarExchange (42) + : : : +- CometHashAggregate (41) + : : : +- CometProject (40) + : : : +- CometBroadcastHashJoin (39) + : : : :- CometProject (37) + : : : : +- CometBroadcastHashJoin (36) + : : : : :- CometFilter (32) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (31) + : : : : +- CometBroadcastExchange (35) + : : : : +- CometFilter (34) + : : : : +- CometScan parquet spark_catalog.default.date_dim (33) + : : : +- ReusedExchange (38) + : : +- CometBroadcastExchange (58) + : : +- CometHashAggregate (57) + : : +- CometColumnarExchange (56) + : : +- CometHashAggregate (55) + : : +- CometProject (54) + : : +- CometBroadcastHashJoin (53) + : : :- CometProject (51) + : : : +- CometBroadcastHashJoin (50) + : : : :- CometFilter (48) + : : : : +- CometScan parquet spark_catalog.default.web_sales (47) + : : : +- ReusedExchange (49) + : : +- ReusedExchange (52) + : +- CometBroadcastExchange (71) + : +- CometHashAggregate (70) + : +- CometColumnarExchange (69) + : +- CometHashAggregate (68) + : +- CometProject (67) + : +- CometBroadcastHashJoin (66) + : :- CometProject (64) + : : +- CometBroadcastHashJoin (63) + : : :- CometFilter (61) + : : : +- CometScan parquet spark_catalog.default.web_sales (60) + : : +- ReusedExchange (62) + : +- ReusedExchange (65) + +- CometBroadcastExchange (85) + +- CometHashAggregate (84) + +- CometColumnarExchange (83) + +- CometHashAggregate (82) + +- CometProject (81) + +- CometBroadcastHashJoin (80) + :- CometProject (78) + : +- CometBroadcastHashJoin (77) + : :- CometFilter (75) + : : +- CometScan parquet spark_catalog.default.web_sales (74) + : +- ReusedExchange (76) + +- ReusedExchange (79) (1) Scan parquet spark_catalog.default.store_sales @@ -161,470 +156,438 @@ 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))] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(15) CometHashAggregate 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))#11] -Results [3]: [ca_county#9, d_year#6, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#11,17,2) AS store_sales#12] -(17) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#13, ss_ext_sales_price#14, ss_sold_date_sk#15] +(16) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13] 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#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#14)] PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct -(18) CometFilter -Input [3]: [ss_addr_sk#13, ss_ext_sales_price#14, ss_sold_date_sk#15] -Condition : isnotnull(ss_addr_sk#13) +(17) CometFilter +Input [3]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13] +Condition : isnotnull(ss_addr_sk#11) -(19) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_qoy#19] +(18) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#15, d_year#16, d_qoy#17] 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) 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)) - -(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] - -(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 - -(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] - -(24) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#20, ca_county#21] - -(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 - -(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] - -(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))] - -(28) ColumnarToRow [codegen id : 2] -Input [4]: [ca_county#21, d_qoy#19, d_year#18, sum#22] - -(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] - -(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] - -(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 - -(33) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] +(19) CometFilter +Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] +Condition : ((((isnotnull(d_qoy#17) AND isnotnull(d_year#16)) AND (d_qoy#17 = 2)) AND (d_year#16 = 2000)) AND isnotnull(d_date_sk#15)) + +(20) CometBroadcastExchange +Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] +Arguments: [d_date_sk#15, d_year#16, d_qoy#17] + +(21) CometBroadcastHashJoin +Left output [3]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13] +Right output [3]: [d_date_sk#15, d_year#16, d_qoy#17] +Arguments: [ss_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight + +(22) CometProject +Input [6]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_year#16, d_qoy#17] +Arguments: [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17], [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17] + +(23) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#18, ca_county#19] + +(24) CometBroadcastHashJoin +Left output [4]: [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17] +Right output [2]: [ca_address_sk#18, ca_county#19] +Arguments: [ss_addr_sk#11], [ca_address_sk#18], Inner, BuildRight + +(25) CometProject +Input [6]: [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_address_sk#18, ca_county#19] +Arguments: [ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_county#19], [ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_county#19] + +(26) CometHashAggregate +Input [4]: [ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_county#19] +Keys [3]: [ca_county#19, d_qoy#17, d_year#16] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#12))] + +(27) CometColumnarExchange +Input [4]: [ca_county#19, d_qoy#17, d_year#16, sum#20] +Arguments: hashpartitioning(ca_county#19, d_qoy#17, d_year#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(28) CometHashAggregate +Input [4]: [ca_county#19, d_qoy#17, d_year#16, sum#20] +Keys [3]: [ca_county#19, d_qoy#17, d_year#16] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#12))] + +(29) CometBroadcastExchange +Input [2]: [ca_county#19, store_sales#21] +Arguments: [ca_county#19, store_sales#21] + +(30) CometBroadcastHashJoin +Left output [3]: [ca_county#9, d_year#6, store_sales#22] +Right output [2]: [ca_county#19, store_sales#21] +Arguments: [ca_county#9], [ca_county#19], Inner, BuildRight + +(31) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25] 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#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct -(34) CometFilter -Input [3]: [ss_addr_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_addr_sk#24) +(32) CometFilter +Input [3]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25] +Condition : isnotnull(ss_addr_sk#23) -(35) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#28, d_year#29, d_qoy#30] +(33) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#27, d_year#28, d_qoy#29] 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 -(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)) +(34) CometFilter +Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] +Condition : ((((isnotnull(d_qoy#29) AND isnotnull(d_year#28)) AND (d_qoy#29 = 3)) AND (d_year#28 = 2000)) AND isnotnull(d_date_sk#27)) + +(35) CometBroadcastExchange +Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] +Arguments: [d_date_sk#27, d_year#28, d_qoy#29] -(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] +(36) CometBroadcastHashJoin +Left output [3]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25] +Right output [3]: [d_date_sk#27, d_year#28, d_qoy#29] +Arguments: [ss_sold_date_sk#25], [d_date_sk#27], Inner, BuildRight -(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 +(37) CometProject +Input [6]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25, d_date_sk#27, d_year#28, d_qoy#29] +Arguments: [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29], [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29] -(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] +(38) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#30, ca_county#31] -(40) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#31, ca_county#32] +(39) CometBroadcastHashJoin +Left output [4]: [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29] +Right output [2]: [ca_address_sk#30, ca_county#31] +Arguments: [ss_addr_sk#23], [ca_address_sk#30], Inner, BuildRight -(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 +(40) CometProject +Input [6]: [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_address_sk#30, ca_county#31] +Arguments: [ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_county#31], [ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_county#31] -(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] +(41) CometHashAggregate +Input [4]: [ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_county#31] +Keys [3]: [ca_county#31, d_qoy#29, d_year#28] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#24))] + +(42) CometColumnarExchange +Input [4]: [ca_county#31, d_qoy#29, d_year#28, sum#32] +Arguments: hashpartitioning(ca_county#31, d_qoy#29, d_year#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (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 - -(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] - -(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] +Input [4]: [ca_county#31, d_qoy#29, d_year#28, sum#32] +Keys [3]: [ca_county#31, d_qoy#29, d_year#28] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#24))] + +(44) CometBroadcastExchange +Input [2]: [ca_county#31, store_sales#33] +Arguments: [ca_county#31, store_sales#33] + +(45) CometBroadcastHashJoin +Left output [5]: [ca_county#9, d_year#6, store_sales#22, ca_county#19, store_sales#21] +Right output [2]: [ca_county#31, store_sales#33] +Arguments: [ca_county#19], [ca_county#31], Inner, BuildRight + +(46) CometProject +Input [7]: [ca_county#9, d_year#6, store_sales#22, ca_county#19, store_sales#21, ca_county#31, store_sales#33] +Arguments: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33], [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33] + +(47) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#37), dynamicpruningexpression(ws_sold_date_sk#37 IN dynamicpruning#38)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#37)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(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) +(48) CometFilter +Input [3]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Condition : isnotnull(ws_bill_addr_sk#34) + +(49) ReusedExchange [Reuses operator id: 5] +Output [3]: [d_date_sk#38, d_year#39, d_qoy#40] + +(50) CometBroadcastHashJoin +Left output [3]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Right output [3]: [d_date_sk#38, d_year#39, d_qoy#40] +Arguments: [ws_sold_date_sk#36], [d_date_sk#38], Inner, BuildRight -(52) ReusedExchange [Reuses operator id: 5] -Output [3]: [d_date_sk#39, d_year#40, d_qoy#41] +(51) CometProject +Input [6]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#38, d_year#39, d_qoy#40] +Arguments: [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40], [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40] + +(52) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#41, ca_county#42] (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 +Left output [4]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40] +Right output [2]: [ca_address_sk#41, ca_county#42] +Arguments: [ws_bill_addr_sk#34], [ca_address_sk#41], Inner, BuildRight (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] - -(55) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#42, ca_county#43] - -(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 - -(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] - -(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#43] -Join type: Inner -Join condition: None - -(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] +Input [6]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_address_sk#41, ca_county#42] +Arguments: [ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_county#42], [ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_county#42] + +(55) CometHashAggregate +Input [4]: [ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_county#42] +Keys [3]: [ca_county#42, d_qoy#40, d_year#39] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] + +(56) CometColumnarExchange +Input [4]: [ca_county#42, d_qoy#40, d_year#39, sum#43] +Arguments: hashpartitioning(ca_county#42, d_qoy#40, d_year#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(57) CometHashAggregate +Input [4]: [ca_county#42, d_qoy#40, d_year#39, sum#43] +Keys [3]: [ca_county#42, d_qoy#40, d_year#39] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] + +(58) CometBroadcastExchange +Input [2]: [ca_county#42, web_sales#44] +Arguments: [ca_county#42, web_sales#44] + +(59) CometBroadcastHashJoin +Left output [5]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33] +Right output [2]: [ca_county#42, web_sales#44] +Arguments: [ca_county#9], [ca_county#42], Inner, BuildRight + +(60) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#49), dynamicpruningexpression(ws_sold_date_sk#49 IN dynamicpruning#50)] +PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#48)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(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) - -(66) ReusedExchange [Reuses operator id: 21] -Output [3]: [d_date_sk#51, d_year#52, d_qoy#53] - -(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 - -(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] - -(69) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#54, ca_county#55] - -(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 - -(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] - -(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#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) - -(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] - -(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] +(61) CometFilter +Input [3]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47] +Condition : isnotnull(ws_bill_addr_sk#45) + +(62) ReusedExchange [Reuses operator id: 20] +Output [3]: [d_date_sk#49, d_year#50, d_qoy#51] + +(63) CometBroadcastHashJoin +Left output [3]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47] +Right output [3]: [d_date_sk#49, d_year#50, d_qoy#51] +Arguments: [ws_sold_date_sk#47], [d_date_sk#49], Inner, BuildRight + +(64) CometProject +Input [6]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47, d_date_sk#49, d_year#50, d_qoy#51] +Arguments: [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51], [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51] + +(65) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#52, ca_county#53] + +(66) CometBroadcastHashJoin +Left output [4]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51] +Right output [2]: [ca_address_sk#52, ca_county#53] +Arguments: [ws_bill_addr_sk#45], [ca_address_sk#52], Inner, BuildRight + +(67) CometProject +Input [6]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_address_sk#52, ca_county#53] +Arguments: [ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_county#53], [ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_county#53] + +(68) CometHashAggregate +Input [4]: [ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_county#53] +Keys [3]: [ca_county#53, d_qoy#51, d_year#50] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#46))] + +(69) CometColumnarExchange +Input [4]: [ca_county#53, d_qoy#51, d_year#50, sum#54] +Arguments: hashpartitioning(ca_county#53, d_qoy#51, d_year#50, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(70) CometHashAggregate +Input [4]: [ca_county#53, d_qoy#51, d_year#50, sum#54] +Keys [3]: [ca_county#53, d_qoy#51, d_year#50] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#46))] + +(71) CometBroadcastExchange +Input [2]: [ca_county#53, web_sales#55] +Arguments: [ca_county#53, web_sales#55] + +(72) CometBroadcastHashJoin +Left output [7]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44] +Right output [2]: [ca_county#53, web_sales#55] +Arguments: [ca_county#42], [ca_county#53], Inner, (CASE WHEN (web_sales#44 > 0.00) THEN (web_sales#55 / web_sales#44) END > CASE WHEN (store_sales#22 > 0.00) THEN (store_sales#21 / store_sales#22) END), BuildRight + +(73) CometProject +Input [9]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, ca_county#53, web_sales#55] +Arguments: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55], [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55] + +(74) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#60), dynamicpruningexpression(ws_sold_date_sk#60 IN dynamicpruning#61)] +PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#59)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(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) +(75) CometFilter +Input [3]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58] +Condition : isnotnull(ws_bill_addr_sk#56) -(81) ReusedExchange [Reuses operator id: 37] -Output [3]: [d_date_sk#62, d_year#63, d_qoy#64] +(76) ReusedExchange [Reuses operator id: 35] +Output [3]: [d_date_sk#60, d_year#61, d_qoy#62] -(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 +(77) CometBroadcastHashJoin +Left output [3]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58] +Right output [3]: [d_date_sk#60, d_year#61, d_qoy#62] +Arguments: [ws_sold_date_sk#58], [d_date_sk#60], Inner, BuildRight -(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] +(78) CometProject +Input [6]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58, d_date_sk#60, d_year#61, d_qoy#62] +Arguments: [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62], [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62] -(84) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#65, ca_county#66] +(79) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#63, ca_county#64] -(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 +(80) CometBroadcastHashJoin +Left output [4]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62] +Right output [2]: [ca_address_sk#63, ca_county#64] +Arguments: [ws_bill_addr_sk#56], [ca_address_sk#63], Inner, BuildRight -(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] +(81) CometProject +Input [6]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_address_sk#63, ca_county#64] +Arguments: [ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_county#64], [ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_county#64] -(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))] +(82) CometHashAggregate +Input [4]: [ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_county#64] +Keys [3]: [ca_county#64, d_qoy#62, d_year#61] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#57))] -(88) ColumnarToRow [codegen id : 10] -Input [4]: [ca_county#66, d_qoy#64, d_year#63, sum#67] +(83) CometColumnarExchange +Input [4]: [ca_county#64, d_qoy#62, d_year#61, sum#65] +Arguments: hashpartitioning(ca_county#64, d_qoy#62, d_year#61, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(89) Exchange -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] +(84) CometHashAggregate +Input [4]: [ca_county#64, d_qoy#62, d_year#61, sum#65] +Keys [3]: [ca_county#64, d_qoy#62, d_year#61] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#57))] -(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] +(85) CometBroadcastExchange +Input [2]: [ca_county#64, web_sales#66] +Arguments: [ca_county#64, web_sales#66] -(91) BroadcastExchange -Input [2]: [ca_county#66, web_sales#68] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] +(86) CometBroadcastHashJoin +Left output [8]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55] +Right output [2]: [ca_county#64, web_sales#66] +Arguments: [ca_county#42], [ca_county#64], Inner, (CASE WHEN (web_sales#55 > 0.00) THEN (web_sales#66 / web_sales#55) END > CASE WHEN (store_sales#21 > 0.00) THEN (store_sales#33 / store_sales#21) END), BuildRight -(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) +(87) CometProject +Input [10]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55, ca_county#64, web_sales#66] +Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70], [ca_county#9, d_year#6, (web_sales#55 / web_sales#44) AS web_q1_q2_increase#67, (store_sales#21 / store_sales#22) AS store_q1_q2_increase#68, (web_sales#66 / web_sales#55) AS web_q2_q3_increase#69, (store_sales#33 / store_sales#21) AS store_q2_q3_increase#70] -(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] +(88) CometColumnarExchange +Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] +Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(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] +(89) CometSort +Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] +Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70], [ca_county#9 ASC NULLS FIRST] -(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 +(90) ColumnarToRow [codegen id : 1] +Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (99) -+- * ColumnarToRow (98) - +- CometFilter (97) - +- CometScan parquet spark_catalog.default.date_dim (96) +BroadcastExchange (94) ++- * ColumnarToRow (93) + +- CometFilter (92) + +- CometScan parquet spark_catalog.default.date_dim (91) -(96) Scan parquet spark_catalog.default.date_dim +(91) 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 -(97) CometFilter +(92) 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)) -(98) ColumnarToRow [codegen id : 1] +(93) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -(99) BroadcastExchange +(94) 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=13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -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) +Subquery:2 Hosting operator id = 16 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 +BroadcastExchange (98) ++- * ColumnarToRow (97) + +- CometFilter (96) + +- CometScan parquet spark_catalog.default.date_dim (95) -(100) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_qoy#19] +(95) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#15, d_year#16, d_qoy#17] 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 -(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)) +(96) CometFilter +Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] +Condition : ((((isnotnull(d_qoy#17) AND isnotnull(d_year#16)) AND (d_qoy#17 = 2)) AND (d_year#16 = 2000)) AND isnotnull(d_date_sk#15)) -(102) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#17, d_year#18, d_qoy#19] +(97) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] -(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] +(98) BroadcastExchange +Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -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) +Subquery:3 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 +BroadcastExchange (102) ++- * ColumnarToRow (101) + +- CometFilter (100) + +- CometScan parquet spark_catalog.default.date_dim (99) -(104) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#28, d_year#29, d_qoy#30] +(99) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#27, d_year#28, d_qoy#29] 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 -(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)) +(100) CometFilter +Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] +Condition : ((((isnotnull(d_qoy#29) AND isnotnull(d_year#28)) AND (d_qoy#29 = 3)) AND (d_year#28 = 2000)) AND isnotnull(d_date_sk#27)) -(106) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#28, d_year#29, d_qoy#30] +(101) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] -(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] +(102) BroadcastExchange +Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -Subquery:4 Hosting operator id = 50 Hosting Expression = ws_sold_date_sk#37 IN dynamicpruning#4 +Subquery:4 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#4 -Subquery:5 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#49 IN dynamicpruning#16 +Subquery:5 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#14 -Subquery:6 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#60 IN dynamicpruning#27 +Subquery:6 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#26 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/simplified.txt index a48c7ad26..6dcf5714d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/simplified.txt @@ -1,150 +1,116 @@ -WholeStageCodegen (13) - Sort [ca_county] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [ca_county] #1 - 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] - BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county] - Project [ca_county,d_year,store_sales,store_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county] - BroadcastHashJoin [ca_county,ca_county] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] - CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] - CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #4 - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [ca_address_sk,ca_county] #5 - CometFilter [ca_address_sk,ca_county] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - InputAdapter - 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] #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] - CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] - CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_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 #8 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #9 + CometSort [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] + CometColumnarExchange [ca_county] #1 + CometProject [web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] + CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales,ca_county,web_sales] + CometProject [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] + CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,ca_county,web_sales] + CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales] + CometProject [ca_county,d_year,store_sales,store_sales,store_sales] + CometBroadcastHashJoin [ca_county,d_year,store_sales,ca_county,store_sales,ca_county,store_sales] + CometBroadcastHashJoin [ca_county,d_year,store_sales,ca_county,store_sales] + CometHashAggregate [ca_county,d_year,store_sales,d_qoy,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange [ca_county,d_qoy,d_year] #2 + CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] + CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #4 + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [ca_address_sk,ca_county] #5 + CometFilter [ca_address_sk,ca_county] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + CometBroadcastExchange [ca_county,store_sales] #6 + CometHashAggregate [ca_county,store_sales,d_qoy,d_year,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange [ca_county,d_qoy,d_year] #7 + CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] + CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_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 #8 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - ReusedExchange [ca_address_sk,ca_county] #5 - InputAdapter - 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] #11 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] - CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] - CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_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 #12 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #13 + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #9 + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + ReusedExchange [ca_address_sk,ca_county] #5 + CometBroadcastExchange [ca_county,store_sales] #10 + CometHashAggregate [ca_county,store_sales,d_qoy,d_year,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange [ca_county,d_qoy,d_year] #11 + CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] + CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_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 #12 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - ReusedExchange [ca_address_sk,ca_county] #5 - InputAdapter - 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] #15 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] - CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] - CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 - ReusedExchange [d_date_sk,d_year,d_qoy] #4 - ReusedExchange [ca_address_sk,ca_county] #5 - InputAdapter - 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] #17 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] - CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] - CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 - ReusedExchange [d_date_sk,d_year,d_qoy] #9 - ReusedExchange [ca_address_sk,ca_county] #5 - InputAdapter - 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] #19 - WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] - CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] - CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 - ReusedExchange [d_date_sk,d_year,d_qoy] #13 - ReusedExchange [ca_address_sk,ca_county] #5 + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #13 + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + ReusedExchange [ca_address_sk,ca_county] #5 + CometBroadcastExchange [ca_county,web_sales] #14 + CometHashAggregate [ca_county,web_sales,d_qoy,d_year,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometColumnarExchange [ca_county,d_qoy,d_year] #15 + CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] + CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 + ReusedExchange [d_date_sk,d_year,d_qoy] #4 + ReusedExchange [ca_address_sk,ca_county] #5 + CometBroadcastExchange [ca_county,web_sales] #16 + CometHashAggregate [ca_county,web_sales,d_qoy,d_year,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometColumnarExchange [ca_county,d_qoy,d_year] #17 + CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] + CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 + ReusedExchange [d_date_sk,d_year,d_qoy] #9 + ReusedExchange [ca_address_sk,ca_county] #5 + CometBroadcastExchange [ca_county,web_sales] #18 + CometHashAggregate [ca_county,web_sales,d_qoy,d_year,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometColumnarExchange [ca_county,d_qoy,d_year] #19 + CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] + CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 + 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-spark3_5/q32/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/explain.txt index b7eee60b1..701a75e24 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/explain.txt @@ -1,35 +1,34 @@ == Physical Plan == -* 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) +* ColumnarToRow (30) ++- CometHashAggregate (29) + +- CometColumnarExchange (28) + +- CometHashAggregate (27) + +- CometProject (26) + +- CometBroadcastHashJoin (25) + :- CometProject (23) + : +- CometBroadcastHashJoin (22) + : :- 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 (21) + : +- CometFilter (20) + : +- CometHashAggregate (19) + : +- CometColumnarExchange (18) + : +- CometHashAggregate (17) + : +- CometProject (16) + : +- CometBroadcastHashJoin (15) + : :- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.catalog_sales (9) + : +- CometBroadcastExchange (14) + : +- CometProject (13) + : +- CometFilter (12) + : +- CometScan parquet spark_catalog.default.date_dim (11) + +- ReusedExchange (24) (1) Scan parquet spark_catalog.default.catalog_sales @@ -72,10 +71,7 @@ Arguments: [cs_item_sk#1], [i_item_sk#5], Inner, BuildRight 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] - -(10) Scan parquet spark_catalog.default.catalog_sales +(9) 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 [] @@ -83,138 +79,130 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(11) CometFilter +(10) CometFilter Input [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] Condition : isnotnull(cs_item_sk#7) -(12) Scan parquet spark_catalog.default.date_dim +(11) 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 -(13) CometFilter +(12) 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)) -(14) CometProject +(13) CometProject Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(15) CometBroadcastExchange +(14) CometBroadcastExchange Input [1]: [d_date_sk#11] Arguments: [d_date_sk#11] -(16) CometBroadcastHashJoin +(15) 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 +(16) 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) CometHashAggregate +(17) 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))] -(19) ColumnarToRow [codegen id : 1] -Input [3]: [cs_item_sk#7, sum#13, count#14] - -(20) Exchange +(18) CometColumnarExchange Input [3]: [cs_item_sk#7, sum#13, count#14] -Arguments: hashpartitioning(cs_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: hashpartitioning(cs_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(21) HashAggregate [codegen id : 2] +(19) CometHashAggregate 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))#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] -(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) +(20) CometFilter +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] +Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#15) -(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] +(21) CometBroadcastExchange +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] +Arguments: [(1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] -(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))#16) +(22) CometBroadcastHashJoin +Left output [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] +Right output [2]: [(1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] +Arguments: [i_item_sk#5], [cs_item_sk#7], Inner, (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#15), BuildRight -(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))#16, cs_item_sk#7] +(23) CometProject +Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] +Arguments: [cs_ext_discount_amt#2, cs_sold_date_sk#3], [cs_ext_discount_amt#2, cs_sold_date_sk#3] -(26) ReusedExchange [Reuses operator id: 36] -Output [1]: [d_date_sk#17] +(24) ReusedExchange [Reuses operator id: 14] +Output [1]: [d_date_sk#16] -(27) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#17] -Join type: Inner -Join condition: None +(25) CometBroadcastHashJoin +Left output [2]: [cs_ext_discount_amt#2, cs_sold_date_sk#3] +Right output [1]: [d_date_sk#16] +Arguments: [cs_sold_date_sk#3], [d_date_sk#16], Inner, BuildRight -(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#17] +(26) CometProject +Input [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, d_date_sk#16] +Arguments: [cs_ext_discount_amt#2], [cs_ext_discount_amt#2] -(29) HashAggregate [codegen id : 4] +(27) CometHashAggregate Input [1]: [cs_ext_discount_amt#2] Keys: [] Functions [1]: [partial_sum(UnscaledValue(cs_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum#18] -Results [1]: [sum#19] -(30) Exchange -Input [1]: [sum#19] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] +(28) CometColumnarExchange +Input [1]: [sum#17] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(31) HashAggregate [codegen id : 5] -Input [1]: [sum#19] +(29) CometHashAggregate +Input [1]: [sum#17] Keys: [] Functions [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))] -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] + +(30) ColumnarToRow [codegen id : 1] +Input [1]: [excess discount amount#18] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (36) -+- * ColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan parquet spark_catalog.default.date_dim (32) +BroadcastExchange (35) ++- * ColumnarToRow (34) + +- CometProject (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.date_dim (31) -(32) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#17, d_date#22] +(31) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#16, d_date#19] 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 -(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) CometFilter +Input [2]: [d_date_sk#16, d_date#19] +Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-01-27)) AND (d_date#19 <= 2000-04-26)) AND isnotnull(d_date_sk#16)) -(34) CometProject -Input [2]: [d_date_sk#17, d_date#22] -Arguments: [d_date_sk#17], [d_date_sk#17] +(33) CometProject +Input [2]: [d_date_sk#16, d_date#19] +Arguments: [d_date_sk#16], [d_date_sk#16] -(35) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#17] +(34) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#16] -(36) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(35) BroadcastExchange +Input [1]: [d_date_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -Subquery:2 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/simplified.txt index c3bfdfd77..19be8718e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/simplified.txt @@ -1,50 +1,41 @@ -WholeStageCodegen (5) - HashAggregate [sum] [sum(UnscaledValue(cs_ext_discount_amt)),excess discount amount,sum] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange #1 - 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))] - ColumnarToRow - InputAdapter - CometProject [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] - CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] - CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk] #3 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - InputAdapter - BroadcastExchange #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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [cs_item_sk,sum,count,cs_ext_discount_amt] - CometProject [cs_item_sk,cs_ext_discount_amt] - CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,d_date_sk] - CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - ReusedExchange [d_date_sk] #2 + CometHashAggregate [excess discount amount,sum,sum(UnscaledValue(cs_ext_discount_amt))] + CometColumnarExchange #1 + CometHashAggregate [sum,cs_ext_discount_amt] + CometProject [cs_ext_discount_amt] + CometBroadcastHashJoin [cs_ext_discount_amt,cs_sold_date_sk,d_date_sk] + CometProject [cs_ext_discount_amt,cs_sold_date_sk] + CometBroadcastHashJoin [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk,(1.3 * avg(cs_ext_discount_amt)),cs_item_sk] + CometProject [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] + CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk] #3 + CometProject [i_item_sk] + CometFilter [i_item_sk,i_manufact_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + CometBroadcastExchange [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk] #4 + CometFilter [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk] + CometHashAggregate [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk,sum,count,avg(UnscaledValue(cs_ext_discount_amt))] + CometColumnarExchange [cs_item_sk] #5 + CometHashAggregate [cs_item_sk,sum,count,cs_ext_discount_amt] + CometProject [cs_item_sk,cs_ext_discount_amt] + CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,d_date_sk] + CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/explain.txt index 8e385ed4e..6fb8b05fb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/explain.txt @@ -1,67 +1,65 @@ == Physical Plan == -TakeOrderedAndProject (63) -+- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- Union (59) - :- * HashAggregate (28) - : +- Exchange (27) - : +- * 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) - : +- * 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) - +- * 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) +* ColumnarToRow (61) ++- CometTakeOrderedAndProject (60) + +- CometHashAggregate (59) + +- CometColumnarExchange (58) + +- CometHashAggregate (57) + +- CometUnion (56) + :- CometHashAggregate (27) + : +- CometColumnarExchange (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) + :- CometHashAggregate (41) + : +- CometColumnarExchange (40) + : +- CometHashAggregate (39) + : +- CometProject (38) + : +- CometBroadcastHashJoin (37) + : :- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometProject (32) + : : : +- CometBroadcastHashJoin (31) + : : : :- CometFilter (29) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (28) + : : : +- ReusedExchange (30) + : : +- ReusedExchange (33) + : +- ReusedExchange (36) + +- CometHashAggregate (55) + +- CometColumnarExchange (54) + +- CometHashAggregate (53) + +- CometProject (52) + +- CometBroadcastHashJoin (51) + :- CometProject (49) + : +- CometBroadcastHashJoin (48) + : :- CometProject (46) + : : +- CometBroadcastHashJoin (45) + : : :- CometFilter (43) + : : : +- CometScan parquet spark_catalog.default.web_sales (42) + : : +- ReusedExchange (44) + : +- ReusedExchange (47) + +- ReusedExchange (50) (1) Scan parquet spark_catalog.default.store_sales @@ -185,212 +183,199 @@ 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))] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarExchange Input [2]: [i_manufact_id#12, sum#15] +Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(27) Exchange -Input [2]: [i_manufact_id#12, sum#15] -Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=1] - -(28) HashAggregate [codegen id : 2] +(27) CometHashAggregate 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))#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#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] +(28) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_bill_addr_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#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#22)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#20)] PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(30) CometFilter -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)) +(29) CometFilter +Input [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] +Condition : (isnotnull(cs_bill_addr_sk#16) AND isnotnull(cs_item_sk#17)) -(31) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#23] +(30) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#21] -(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 +(31) CometBroadcastHashJoin +Left output [4]: [cs_bill_addr_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 -(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] +(32) CometProject +Input [5]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19, d_date_sk#21] +Arguments: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18], [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18] -(34) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#24] +(33) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#22] -(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 +(34) CometBroadcastHashJoin +Left output [3]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18] +Right output [1]: [ca_address_sk#22] +Arguments: [cs_bill_addr_sk#16], [ca_address_sk#22], 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] +(35) CometProject +Input [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, ca_address_sk#22] +Arguments: [cs_item_sk#17, cs_ext_sales_price#18], [cs_item_sk#17, cs_ext_sales_price#18] -(37) ReusedExchange [Reuses operator id: 22] -Output [2]: [i_item_sk#25, i_manufact_id#26] +(36) ReusedExchange [Reuses operator id: 22] +Output [2]: [i_item_sk#23, i_manufact_id#24] -(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 +(37) CometBroadcastHashJoin +Left output [2]: [cs_item_sk#17, cs_ext_sales_price#18] +Right output [2]: [i_item_sk#23, i_manufact_id#24] +Arguments: [cs_item_sk#17], [i_item_sk#23], 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] +(38) CometProject +Input [4]: [cs_item_sk#17, cs_ext_sales_price#18, i_item_sk#23, i_manufact_id#24] +Arguments: [cs_ext_sales_price#18, i_manufact_id#24], [cs_ext_sales_price#18, i_manufact_id#24] -(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))] +(39) CometHashAggregate +Input [2]: [cs_ext_sales_price#18, i_manufact_id#24] +Keys [1]: [i_manufact_id#24] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#18))] -(41) ColumnarToRow [codegen id : 3] -Input [2]: [i_manufact_id#26, sum#27] +(40) CometColumnarExchange +Input [2]: [i_manufact_id#24, sum#25] +Arguments: hashpartitioning(i_manufact_id#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(42) Exchange -Input [2]: [i_manufact_id#26, sum#27] -Arguments: hashpartitioning(i_manufact_id#26, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(41) CometHashAggregate +Input [2]: [i_manufact_id#24, sum#25] +Keys [1]: [i_manufact_id#24] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#18))] -(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] +(42) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] 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#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#30)] 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)) +(43) CometFilter +Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] +Condition : (isnotnull(ws_bill_addr_sk#27) AND isnotnull(ws_item_sk#26)) -(46) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#35] +(44) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#31] -(47) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] -Right output [1]: [d_date_sk#35] -Arguments: [ws_sold_date_sk#33], [d_date_sk#35], Inner, BuildRight +(45) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] +Right output [1]: [d_date_sk#31] +Arguments: [ws_sold_date_sk#29], [d_date_sk#31], Inner, BuildRight -(48) CometProject -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] -Arguments: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32], [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] +(46) CometProject +Input [5]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29, d_date_sk#31] +Arguments: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28], [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28] -(49) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#36] +(47) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#32] -(50) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] -Right output [1]: [ca_address_sk#36] -Arguments: [ws_bill_addr_sk#31], [ca_address_sk#36], Inner, BuildRight +(48) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28] +Right output [1]: [ca_address_sk#32] +Arguments: [ws_bill_addr_sk#27], [ca_address_sk#32], Inner, BuildRight -(51) CometProject -Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ca_address_sk#36] -Arguments: [ws_item_sk#30, ws_ext_sales_price#32], [ws_item_sk#30, ws_ext_sales_price#32] +(49) CometProject +Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ca_address_sk#32] +Arguments: [ws_item_sk#26, ws_ext_sales_price#28], [ws_item_sk#26, ws_ext_sales_price#28] -(52) ReusedExchange [Reuses operator id: 22] -Output [2]: [i_item_sk#37, i_manufact_id#38] +(50) ReusedExchange [Reuses operator id: 22] +Output [2]: [i_item_sk#33, i_manufact_id#34] -(53) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_ext_sales_price#32] -Right output [2]: [i_item_sk#37, i_manufact_id#38] -Arguments: [ws_item_sk#30], [i_item_sk#37], Inner, BuildRight +(51) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#26, ws_ext_sales_price#28] +Right output [2]: [i_item_sk#33, i_manufact_id#34] +Arguments: [ws_item_sk#26], [i_item_sk#33], Inner, BuildRight -(54) CometProject -Input [4]: [ws_item_sk#30, ws_ext_sales_price#32, i_item_sk#37, i_manufact_id#38] -Arguments: [ws_ext_sales_price#32, i_manufact_id#38], [ws_ext_sales_price#32, i_manufact_id#38] +(52) CometProject +Input [4]: [ws_item_sk#26, ws_ext_sales_price#28, i_item_sk#33, i_manufact_id#34] +Arguments: [ws_ext_sales_price#28, i_manufact_id#34], [ws_ext_sales_price#28, i_manufact_id#34] -(55) CometHashAggregate -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))] +(53) CometHashAggregate +Input [2]: [ws_ext_sales_price#28, i_manufact_id#34] +Keys [1]: [i_manufact_id#34] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#28))] -(56) ColumnarToRow [codegen id : 5] -Input [2]: [i_manufact_id#38, sum#39] +(54) CometColumnarExchange +Input [2]: [i_manufact_id#34, sum#35] +Arguments: hashpartitioning(i_manufact_id#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(57) Exchange -Input [2]: [i_manufact_id#38, sum#39] -Arguments: hashpartitioning(i_manufact_id#38, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(58) HashAggregate [codegen id : 6] -Input [2]: [i_manufact_id#38, sum#39] -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))#40] -Results [2]: [i_manufact_id#38, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#32))#40,17,2) AS total_sales#41] +(55) CometHashAggregate +Input [2]: [i_manufact_id#34, sum#35] +Keys [1]: [i_manufact_id#34] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#28))] -(59) Union +(56) CometUnion +Child 0 Input [2]: [i_manufact_id#12, total_sales#36] +Child 1 Input [2]: [i_manufact_id#24, total_sales#37] +Child 2 Input [2]: [i_manufact_id#34, total_sales#38] -(60) HashAggregate [codegen id : 7] -Input [2]: [i_manufact_id#12, total_sales#17] +(57) CometHashAggregate +Input [2]: [i_manufact_id#12, total_sales#36] Keys [1]: [i_manufact_id#12] -Functions [1]: [partial_sum(total_sales#17)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [3]: [i_manufact_id#12, sum#44, isEmpty#45] +Functions [1]: [partial_sum(total_sales#36)] -(61) Exchange -Input [3]: [i_manufact_id#12, sum#44, isEmpty#45] -Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(58) CometColumnarExchange +Input [3]: [i_manufact_id#12, sum#39, isEmpty#40] +Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(62) HashAggregate [codegen id : 8] -Input [3]: [i_manufact_id#12, sum#44, isEmpty#45] +(59) CometHashAggregate +Input [3]: [i_manufact_id#12, sum#39, isEmpty#40] Keys [1]: [i_manufact_id#12] -Functions [1]: [sum(total_sales#17)] -Aggregate Attributes [1]: [sum(total_sales#17)#46] -Results [2]: [i_manufact_id#12, sum(total_sales#17)#46 AS total_sales#47] +Functions [1]: [sum(total_sales#36)] + +(60) CometTakeOrderedAndProject +Input [2]: [i_manufact_id#12, total_sales#41] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_sales#41 ASC NULLS FIRST], output=[i_manufact_id#12,total_sales#41]), [i_manufact_id#12, total_sales#41], 100, [total_sales#41 ASC NULLS FIRST], [i_manufact_id#12, total_sales#41] -(63) TakeOrderedAndProject -Input [2]: [i_manufact_id#12, total_sales#47] -Arguments: 100, [total_sales#47 ASC NULLS FIRST], [i_manufact_id#12, total_sales#47] +(61) ColumnarToRow [codegen id : 1] +Input [2]: [i_manufact_id#12, total_sales#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (68) -+- * ColumnarToRow (67) - +- CometProject (66) - +- CometFilter (65) - +- CometScan parquet spark_catalog.default.date_dim (64) +BroadcastExchange (66) ++- * ColumnarToRow (65) + +- CometProject (64) + +- CometFilter (63) + +- CometScan parquet spark_catalog.default.date_dim (62) -(64) Scan parquet spark_catalog.default.date_dim +(62) Scan parquet spark_catalog.default.date_dim 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 +(63) 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)) -(66) CometProject +(64) CometProject 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] +(65) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(68) BroadcastExchange +(66) BroadcastExchange Input [1]: [d_date_sk#6] 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#21 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 28 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/simplified.txt index 7a3c2c153..964f250ba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/simplified.txt @@ -1,89 +1,73 @@ -TakeOrderedAndProject [total_sales,i_manufact_id] - WholeStageCodegen (8) - HashAggregate [i_manufact_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - InputAdapter - Exchange [i_manufact_id] #1 - WholeStageCodegen (7) - HashAggregate [i_manufact_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (2) - HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_manufact_id] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_manufact_id,sum,ss_ext_sales_price] - CometProject [ss_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_manufact_id] - CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_manufact_id,total_sales] + CometHashAggregate [i_manufact_id,total_sales,sum,isEmpty,sum(total_sales)] + CometColumnarExchange [i_manufact_id] #1 + CometHashAggregate [i_manufact_id,sum,isEmpty,total_sales] + CometUnion [i_manufact_id,total_sales] + CometHashAggregate [i_manufact_id,total_sales,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange [i_manufact_id] #2 + CometHashAggregate [i_manufact_id,sum,ss_ext_sales_price] + CometProject [ss_ext_sales_price,i_manufact_id] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_manufact_id] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ca_address_sk] #5 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk,i_manufact_id] #6 - CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id] - CometFilter [i_item_sk,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange [i_manufact_id] #7 - CometProject [i_manufact_id] - CometFilter [i_category,i_manufact_id] - 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] #8 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometHashAggregate [i_manufact_id,sum,cs_ext_sales_price] - CometProject [cs_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_manufact_id] - CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] - CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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 - 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] #9 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometHashAggregate [i_manufact_id,sum,ws_ext_sales_price] - CometProject [ws_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_manufact_id] - CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] - CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_manufact_id] #6 + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ca_address_sk] #5 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometBroadcastExchange [i_item_sk,i_manufact_id] #6 + CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id] + CometFilter [i_item_sk,i_manufact_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + CometBroadcastExchange [i_manufact_id] #7 + CometProject [i_manufact_id] + CometFilter [i_category,i_manufact_id] + CometScan parquet spark_catalog.default.item [i_category,i_manufact_id] + CometHashAggregate [i_manufact_id,total_sales,sum,sum(UnscaledValue(cs_ext_sales_price))] + CometColumnarExchange [i_manufact_id] #8 + CometHashAggregate [i_manufact_id,sum,cs_ext_sales_price] + CometProject [cs_ext_sales_price,i_manufact_id] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_manufact_id] + CometProject [cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] + CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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 + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_manufact_id] #6 + CometHashAggregate [i_manufact_id,total_sales,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometColumnarExchange [i_manufact_id] #9 + CometHashAggregate [i_manufact_id,sum,ws_ext_sales_price] + CometProject [ws_ext_sales_price,i_manufact_id] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_manufact_id] + CometProject [ws_item_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] + CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 + 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-spark3_5/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/explain.txt index 5609dc11c..c33735fa4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/explain.txt @@ -1,37 +1,36 @@ == Physical Plan == -* 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) +* ColumnarToRow (32) ++- CometSort (31) + +- CometColumnarExchange (30) + +- CometProject (29) + +- CometBroadcastHashJoin (28) + :- CometFilter (24) + : +- CometHashAggregate (23) + : +- CometColumnarExchange (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) + +- CometBroadcastExchange (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.customer (25) (1) Scan parquet spark_catalog.default.store_sales @@ -135,90 +134,84 @@ 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)] -(22) ColumnarToRow [codegen id : 1] +(22) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(24) HashAggregate [codegen id : 3] +(23) CometHashAggregate 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)#17] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#17 AS cnt#18] -(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)) +(24) CometFilter +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] +Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) -(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] +(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] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(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) +(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) -(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] +(27) CometBroadcastExchange +Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Arguments: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -(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] +(28) CometBroadcastHashJoin +Left output [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] +Right output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight -(30) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#19] -Join type: Inner -Join condition: None +(29) CometProject +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] +Arguments: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17], [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -(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] +(30) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] -(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] +(31) CometSort +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, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17], [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] -(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 +(32) ColumnarToRow [codegen id : 1] +Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (38) -+- * ColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.date_dim (34) +BroadcastExchange (37) ++- * ColumnarToRow (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) -(34) Scan parquet spark_catalog.default.date_dim +(33) 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 -(35) CometFilter +(34) 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)) -(36) CometProject +(35) CometProject Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(37) ColumnarToRow [codegen id : 1] +(36) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(38) BroadcastExchange +(37) BroadcastExchange Input [1]: [d_date_sk#7] -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-spark3_5/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/simplified.txt index e1f7f9bd2..c131bd9c0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/simplified.txt @@ -1,50 +1,42 @@ -WholeStageCodegen (4) - Sort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,count] - CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] - CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] - CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #5 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange [hd_demo_sk] #6 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 + CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometFilter [ss_ticket_number,ss_customer_sk,cnt] + CometHashAggregate [ss_ticket_number,ss_customer_sk,cnt,count,count(1)] + CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] + CometProject [ss_customer_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange [s_store_sk] #5 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_county] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county] + CometBroadcastExchange [hd_demo_sk] #6 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] #7 + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/explain.txt index c06c1dd16..b35338d18 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/explain.txt @@ -1,48 +1,50 @@ == Physical Plan == -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) +TakeOrderedAndProject (46) ++- * HashAggregate (45) + +- * ColumnarToRow (44) + +- CometColumnarExchange (43) + +- RowToColumnar (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 @@ -238,50 +240,56 @@ Functions [10]: [partial_count(1), partial_min(cd_dep_count#25), partial_max(cd_ 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] -(42) Exchange +(42) RowToColumnar 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] -(43) HashAggregate [codegen id : 6] +(43) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=5] + +(44) ColumnarToRow [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] + +(45) 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] -(44) TakeOrderedAndProject +(46) 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 = 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) +BroadcastExchange (51) ++- * ColumnarToRow (50) + +- CometProject (49) + +- CometFilter (48) + +- CometScan parquet spark_catalog.default.date_dim (47) -(45) Scan parquet spark_catalog.default.date_dim +(47) 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 -(46) CometFilter +(48) 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)) -(47) CometProject +(49) CometProject Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(48) ColumnarToRow [codegen id : 1] +(50) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(49) BroadcastExchange +(51) BroadcastExchange Input [1]: [d_date_sk#9] 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-spark3_5/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/simplified.txt index 9deb51342..f9ef660d5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/simplified.txt @@ -1,70 +1,72 @@ 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 (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 (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] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] - CometFilter [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] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,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 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + RowToColumnar + 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] + Project [c_current_cdemo_sk,ca_state] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] + Filter [exists,exists] + BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,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 + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [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] + CometBroadcastExchange [ss_customer_sk] #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,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 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [cs_ship_customer_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,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 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) + BroadcastExchange #8 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - 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] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + 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-v1_4-spark3_5/q36/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/explain.txt index 33b572ff6..9f22e37d6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/explain.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject (29) +- * Project (28) +- Window (27) - +- * Sort (26) - +- Exchange (25) - +- * HashAggregate (24) - +- Exchange (23) - +- * ColumnarToRow (22) + +- * ColumnarToRow (26) + +- CometSort (25) + +- CometColumnarExchange (24) + +- CometHashAggregate (23) + +- CometColumnarExchange (22) +- CometHashAggregate (21) +- CometExpand (20) +- CometProject (19) @@ -131,39 +131,37 @@ Input [5]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#14, i_class#15, sp 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))] -(22) ColumnarToRow [codegen id : 1] +(22) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(24) HashAggregate [codegen id : 2] +(23) CometHashAggregate 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))#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] -(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] +(24) CometColumnarExchange +Input [7]: [gross_margin#19, i_category#14, i_class#15, lochierarchy#20, _w0#21, _w1#22, _w2#23] +Arguments: hashpartitioning(_w1#22, _w2#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(25) CometSort +Input [7]: [gross_margin#19, i_category#14, i_class#15, lochierarchy#20, _w0#21, _w1#22, _w2#23] +Arguments: [gross_margin#19, i_category#14, i_class#15, lochierarchy#20, _w0#21, _w1#22, _w2#23], [_w1#22 ASC NULLS FIRST, _w2#23 ASC NULLS FIRST, _w0#21 ASC NULLS FIRST] -(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) ColumnarToRow [codegen id : 1] +Input [7]: [gross_margin#19, i_category#14, i_class#15, lochierarchy#20, _w0#21, _w1#22, _w2#23] (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] +Input [7]: [gross_margin#19, i_category#14, i_class#15, lochierarchy#20, _w0#21, _w1#22, _w2#23] +Arguments: [rank(_w0#21) windowspecdefinition(_w1#22, _w2#23, _w0#21 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#24], [_w1#22, _w2#23], [_w0#21 ASC NULLS FIRST] -(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) Project [codegen id : 2] +Output [5]: [gross_margin#19, i_category#14, i_class#15, lochierarchy#20, rank_within_parent#24] +Input [8]: [gross_margin#19, i_category#14, i_class#15, lochierarchy#20, _w0#21, _w1#22, _w2#23, rank_within_parent#24] (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] +Input [5]: [gross_margin#19, i_category#14, i_class#15, lochierarchy#20, rank_within_parent#24] +Arguments: 100, [lochierarchy#20 DESC NULLS LAST, CASE WHEN (lochierarchy#20 = 0) THEN i_category#14 END ASC NULLS FIRST, rank_within_parent#24 ASC NULLS FIRST], [gross_margin#19, i_category#14, i_class#15, lochierarchy#20, rank_within_parent#24] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/simplified.txt index 4ad9b501b..d7beb4422 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/simplified.txt @@ -1,45 +1,41 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (4) + WholeStageCodegen (2) Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [_w0,_w1,_w2] - WholeStageCodegen (3) - Sort [_w1,_w2,_w0] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [_w1,_w2] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_category,i_class,spark_grouping_id,sum,sum,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,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] - CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] - CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometFilter [i_item_sk,i_class,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #6 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] + CometColumnarExchange [_w1,_w2] #1 + CometHashAggregate [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2,spark_grouping_id,sum,sum,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange [i_category,i_class,spark_grouping_id] #2 + CometHashAggregate [i_category,i_class,spark_grouping_id,sum,sum,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,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] + CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometFilter [i_item_sk,i_class,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometBroadcastExchange [s_store_sk] #6 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + 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-spark3_5/q37/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/explain.txt index 6eb0f14b1..42dab0b38 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (26) -+- * HashAggregate (25) - +- Exchange (24) - +- * ColumnarToRow (23) +* ColumnarToRow (26) ++- CometTakeOrderedAndProject (25) + +- CometHashAggregate (24) + +- CometColumnarExchange (23) +- CometHashAggregate (22) +- CometProject (21) +- CometBroadcastHashJoin (20) @@ -132,23 +132,21 @@ 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: [] -(23) ColumnarToRow [codegen id : 1] +(23) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(24) 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] -(26) TakeOrderedAndProject +(25) CometTakeOrderedAndProject +Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#2 ASC NULLS FIRST], output=[i_item_id#2,i_item_desc#3,i_current_price#4]), [i_item_id#2, i_item_desc#3, i_current_price#4], 100, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4] + +(26) ColumnarToRow [codegen id : 1] 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 ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/simplified.txt index efd86d5ab..39edfb519 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/simplified.txt @@ -1,38 +1,36 @@ -TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - 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 (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,i_item_id,i_item_desc,i_current_price,cs_item_sk] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2 - CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,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,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] - CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 - CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_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_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] + CometHashAggregate [i_item_id,i_item_desc,i_current_price] + CometColumnarExchange [i_item_id,i_item_desc,i_current_price] #1 + CometHashAggregate [i_item_id,i_item_desc,i_current_price] + CometProject [i_item_id,i_item_desc,i_current_price] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,cs_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2 + CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,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,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] + CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] + CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 + CometProject [inv_item_sk,inv_date_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_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_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [cs_item_sk] + CometFilter [cs_item_sk,cs_sold_date_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-spark3_5/q38/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/explain.txt index b6b11827d..091883bd6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/explain.txt @@ -1,53 +1,51 @@ == Physical Plan == -* 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) +* ColumnarToRow (47) ++- CometHashAggregate (46) + +- CometColumnarExchange (45) + +- CometHashAggregate (44) + +- CometProject (43) + +- CometBroadcastHashJoin (42) + :- CometBroadcastHashJoin (29) + : :- CometHashAggregate (16) + : : +- CometColumnarExchange (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) + : +- CometBroadcastExchange (28) + : +- CometHashAggregate (27) + : +- CometColumnarExchange (26) + : +- CometHashAggregate (25) + : +- CometProject (24) + : +- CometBroadcastHashJoin (23) + : :- CometProject (21) + : : +- CometBroadcastHashJoin (20) + : : :- CometFilter (18) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (17) + : : +- ReusedExchange (19) + : +- ReusedExchange (22) + +- CometBroadcastExchange (41) + +- CometHashAggregate (40) + +- CometColumnarExchange (39) + +- CometHashAggregate (38) + +- CometProject (37) + +- CometBroadcastHashJoin (36) + :- CometProject (34) + : +- CometBroadcastHashJoin (33) + : :- CometFilter (31) + : : +- CometScan parquet spark_catalog.default.web_sales (30) + : +- ReusedExchange (32) + +- ReusedExchange (35) (1) Scan parquet spark_catalog.default.store_sales @@ -119,21 +117,16 @@ 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: [] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(17) HashAggregate [codegen id : 6] +(16) 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#9, c_first_name#8, d_date#5] -(18) Scan parquet spark_catalog.default.catalog_sales +(17) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_bill_customer_sk#10, cs_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] @@ -141,64 +134,58 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#11), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(19) CometFilter +(18) CometFilter Input [2]: [cs_bill_customer_sk#10, cs_sold_date_sk#11] Condition : isnotnull(cs_bill_customer_sk#10) -(20) ReusedExchange [Reuses operator id: 6] +(19) ReusedExchange [Reuses operator id: 6] Output [2]: [d_date_sk#13, d_date#14] -(21) CometBroadcastHashJoin +(20) 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 -(22) CometProject +(21) 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] -(23) ReusedExchange [Reuses operator id: 11] +(22) ReusedExchange [Reuses operator id: 11] Output [3]: [c_customer_sk#15, c_first_name#16, c_last_name#17] -(24) CometBroadcastHashJoin +(23) 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 -(25) CometProject +(24) 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] -(26) CometHashAggregate +(25) 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: [] -(27) ColumnarToRow [codegen id : 2] +(26) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=2] -(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] +(27) 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#17, c_first_name#16, d_date#14] -(30) BroadcastExchange +(28) CometBroadcastExchange 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] +Arguments: [c_last_name#17, c_first_name#16, d_date#14] -(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 +(29) CometBroadcastHashJoin +Left output [3]: [c_last_name#9, c_first_name#8, d_date#5] +Right output [3]: [c_last_name#17, c_first_name#16, d_date#14] +Arguments: [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)], [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)], LeftSemi, BuildRight -(32) Scan parquet spark_catalog.default.web_sales +(30) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] @@ -206,119 +193,111 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#19), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(33) CometFilter +(31) CometFilter Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] Condition : isnotnull(ws_bill_customer_sk#18) -(34) ReusedExchange [Reuses operator id: 6] +(32) ReusedExchange [Reuses operator id: 6] Output [2]: [d_date_sk#21, d_date#22] -(35) CometBroadcastHashJoin +(33) 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 -(36) CometProject +(34) 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] -(37) ReusedExchange [Reuses operator id: 11] +(35) ReusedExchange [Reuses operator id: 11] Output [3]: [c_customer_sk#23, c_first_name#24, c_last_name#25] -(38) CometBroadcastHashJoin +(36) 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 -(39) CometProject +(37) 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] -(40) CometHashAggregate +(38) 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: [] -(41) ColumnarToRow [codegen id : 4] +(39) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=3] -(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] - -(43) HashAggregate [codegen id : 5] +(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#25, c_first_name#24, d_date#22] -(44) BroadcastExchange +(41) CometBroadcastExchange 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] +Arguments: [c_last_name#25, c_first_name#24, d_date#22] -(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 +(42) CometBroadcastHashJoin +Left output [3]: [c_last_name#9, c_first_name#8, d_date#5] +Right output [3]: [c_last_name#25, c_first_name#24, d_date#22] +Arguments: [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)], [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)], LeftSemi, BuildRight -(46) Project [codegen id : 6] -Output: [] +(43) CometProject Input [3]: [c_last_name#9, c_first_name#8, d_date#5] -(47) HashAggregate [codegen id : 6] +(44) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#26] -Results [1]: [count#27] -(48) Exchange -Input [1]: [count#27] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +(45) CometColumnarExchange +Input [1]: [count#26] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(49) HashAggregate [codegen id : 7] -Input [1]: [count#27] +(46) CometHashAggregate +Input [1]: [count#26] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#28] -Results [1]: [count(1)#28 AS count(1)#29] + +(47) ColumnarToRow [codegen id : 1] +Input [1]: [count(1)#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 -BroadcastExchange (54) -+- * ColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan parquet spark_catalog.default.date_dim (50) +BroadcastExchange (52) ++- * ColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) -(50) Scan parquet spark_catalog.default.date_dim +(48) 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 -(51) CometFilter +(49) 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)) -(52) CometProject +(50) 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] -(53) ColumnarToRow [codegen id : 1] +(51) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#4, d_date#5] -(54) BroadcastExchange +(52) BroadcastExchange Input [2]: [d_date_sk#4, d_date#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=5] -Subquery:2 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#11 IN dynamicpruning#3 +Subquery:2 Hosting operator id = 17 Hosting Expression = cs_sold_date_sk#11 IN dynamicpruning#3 -Subquery:3 Hosting operator id = 32 Hosting Expression = ws_sold_date_sk#19 IN dynamicpruning#3 +Subquery:3 Hosting operator id = 30 Hosting Expression = ws_sold_date_sk#19 IN dynamicpruning#3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/simplified.txt index 9d667265c..77970e8d3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/simplified.txt @@ -1,75 +1,59 @@ -WholeStageCodegen (7) - HashAggregate [count] [count(1),count(1),count] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (6) - HashAggregate [count,count] - Project - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #2 - 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,d_date,c_customer_sk,c_first_name,c_last_name] - CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_customer_sk,ss_sold_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,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #4 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,d_date] - 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,d_date,c_customer_sk,c_first_name,c_last_name] - CometProject [cs_bill_customer_sk,d_date] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] - CometFilter [cs_bill_customer_sk,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_date] #4 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - HashAggregate [c_last_name,c_first_name,d_date] - 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,d_date,c_customer_sk,c_first_name,c_last_name] - CometProject [ws_bill_customer_sk,d_date] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_bill_customer_sk,ws_sold_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,d_date] #4 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 + CometHashAggregate [count(1),count,count(1)] + CometColumnarExchange #1 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #2 + CometHashAggregate [c_last_name,c_first_name,d_date] + CometProject [c_last_name,c_first_name,d_date] + CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] + CometProject [ss_customer_sk,d_date] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_customer_sk,ss_sold_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,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange [d_date_sk,d_date] #4 + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 + CometFilter [c_customer_sk,c_first_name,c_last_name] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] + CometBroadcastExchange [c_last_name,c_first_name,d_date] #6 + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #7 + CometHashAggregate [c_last_name,c_first_name,d_date] + CometProject [c_last_name,c_first_name,d_date] + CometBroadcastHashJoin [cs_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] + CometProject [cs_bill_customer_sk,d_date] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] + CometFilter [cs_bill_customer_sk,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_date] #4 + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 + CometBroadcastExchange [c_last_name,c_first_name,d_date] #8 + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #9 + CometHashAggregate [c_last_name,c_first_name,d_date] + CometProject [c_last_name,c_first_name,d_date] + CometBroadcastHashJoin [ws_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] + CometProject [ws_bill_customer_sk,d_date] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_bill_customer_sk,ws_sold_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,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-spark3_5/q39a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/explain.txt index 6b6272590..60c02c542 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/explain.txt @@ -1,52 +1,51 @@ == Physical Plan == -* 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) +* ColumnarToRow (47) ++- CometSort (46) + +- CometColumnarExchange (45) + +- CometBroadcastHashJoin (44) + :- CometProject (23) + : +- CometFilter (22) + : +- CometHashAggregate (21) + : +- CometColumnarExchange (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) + +- CometBroadcastExchange (43) + +- CometProject (42) + +- CometFilter (41) + +- CometHashAggregate (40) + +- CometColumnarExchange (39) + +- CometHashAggregate (38) + +- CometProject (37) + +- CometBroadcastHashJoin (36) + :- CometProject (31) + : +- CometBroadcastHashJoin (30) + : :- CometProject (28) + : : +- CometBroadcastHashJoin (27) + : : :- CometFilter (25) + : : : +- CometScan parquet spark_catalog.default.inventory (24) + : : +- ReusedExchange (26) + : +- ReusedExchange (29) + +- CometBroadcastExchange (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan parquet spark_catalog.default.date_dim (32) (1) Scan parquet spark_catalog.default.inventory @@ -142,197 +141,189 @@ Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_n 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)] -(20) ColumnarToRow [codegen id : 1] +(20) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(22) HashAggregate [codegen id : 4] +(21) CometHashAggregate 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))#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] -(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 +(22) CometFilter +Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#17, mean#18] +Condition : CASE WHEN (mean#18 = 0.0) THEN false ELSE ((stdev#17 / mean#18) > 1.0) END -(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] +(23) CometProject +Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#17, mean#18] +Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19], [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, CASE WHEN (mean#18 = 0.0) THEN null ELSE (stdev#17 / mean#18) END AS cov#19] -(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] +(24) Scan parquet spark_catalog.default.inventory +Output [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#25), dynamicpruningexpression(inv_date_sk#25 IN dynamicpruning#26)] +PartitionFilters: [isnotnull(inv_date_sk#23), dynamicpruningexpression(inv_date_sk#23 IN dynamicpruning#24)] PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(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)) +(25) CometFilter +Input [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] +Condition : (isnotnull(inv_item_sk#20) AND isnotnull(inv_warehouse_sk#21)) -(27) ReusedExchange [Reuses operator id: 5] -Output [1]: [i_item_sk#27] +(26) ReusedExchange [Reuses operator id: 5] +Output [1]: [i_item_sk#25] -(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 +(27) CometBroadcastHashJoin +Left output [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] +Right output [1]: [i_item_sk#25] +Arguments: [inv_item_sk#20], [i_item_sk#25], Inner, BuildRight -(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] +(28) CometProject +Input [5]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] +Arguments: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25], [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] -(30) ReusedExchange [Reuses operator id: 10] -Output [2]: [w_warehouse_sk#28, w_warehouse_name#29] +(29) ReusedExchange [Reuses operator id: 10] +Output [2]: [w_warehouse_sk#26, w_warehouse_name#27] -(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 +(30) CometBroadcastHashJoin +Left output [4]: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] +Right output [2]: [w_warehouse_sk#26, w_warehouse_name#27] +Arguments: [inv_warehouse_sk#21], [w_warehouse_sk#26], Inner, BuildRight -(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] +(31) CometProject +Input [6]: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] +Arguments: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27], [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] -(33) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#30, d_year#31, d_moy#32] +(32) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#28, d_year#29, d_moy#30] 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 -(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)) +(33) CometFilter +Input [3]: [d_date_sk#28, d_year#29, d_moy#30] +Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2001)) AND (d_moy#30 = 2)) AND isnotnull(d_date_sk#28)) -(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] +(34) CometProject +Input [3]: [d_date_sk#28, d_year#29, d_moy#30] +Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] -(36) CometBroadcastExchange -Input [2]: [d_date_sk#30, d_moy#32] -Arguments: [d_date_sk#30, d_moy#32] +(35) CometBroadcastExchange +Input [2]: [d_date_sk#28, d_moy#30] +Arguments: [d_date_sk#28, d_moy#30] -(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 +(36) CometBroadcastHashJoin +Left output [5]: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] +Right output [2]: [d_date_sk#28, d_moy#30] +Arguments: [inv_date_sk#23], [d_date_sk#28], 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] +(37) CometProject +Input [7]: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_date_sk#28, d_moy#30] +Arguments: [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30], [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30] -(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)] +(38) CometHashAggregate +Input [5]: [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30] +Keys [4]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#22 as double)), partial_avg(inv_quantity_on_hand#22)] -(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] +(39) CometColumnarExchange +Input [9]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, n#31, avg#32, m2#33, sum#34, count#35] +Arguments: hashpartitioning(w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(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] +(40) CometHashAggregate +Input [9]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, n#31, avg#32, m2#33, sum#34, count#35] +Keys [4]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#22 as double)), avg(inv_quantity_on_hand#22)] -(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] +(41) CometFilter +Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, stdev#17, mean#18] +Condition : CASE WHEN (mean#18 = 0.0) THEN false ELSE ((stdev#17 / mean#18) > 1.0) END -(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 +(42) CometProject +Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, stdev#17, mean#18] +Arguments: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37], [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#18 AS mean#36, CASE WHEN (mean#18 = 0.0) THEN null ELSE (stdev#17 / mean#18) END AS cov#37] -(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] +(43) CometBroadcastExchange +Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] +Arguments: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -(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] +(44) CometBroadcastHashJoin +Left output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19] +Right output [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] +Arguments: [i_item_sk#6, w_warehouse_sk#7], [i_item_sk#25, w_warehouse_sk#26], Inner, BuildRight -(46) BroadcastHashJoin [codegen id : 4] -Left keys [2]: [i_item_sk#6, w_warehouse_sk#7] -Right keys [2]: [i_item_sk#27, w_warehouse_sk#28] -Join type: Inner -Join condition: None +(45) CometColumnarExchange +Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] +Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(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] +(46) CometSort +Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] +Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST] -(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 +(47) ColumnarToRow [codegen id : 1] +Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (53) -+- * ColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometScan parquet spark_catalog.default.date_dim (49) +BroadcastExchange (52) ++- * ColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) -(49) Scan parquet spark_catalog.default.date_dim +(48) 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 -(50) CometFilter +(49) 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)) -(51) CometProject +(50) 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] -(52) ColumnarToRow [codegen id : 1] +(51) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_moy#11] -(53) BroadcastExchange +(52) BroadcastExchange Input [2]: [d_date_sk#9, d_moy#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] -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) +Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 +BroadcastExchange (57) ++- * ColumnarToRow (56) + +- CometProject (55) + +- CometFilter (54) + +- CometScan parquet spark_catalog.default.date_dim (53) -(54) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#30, d_year#31, d_moy#32] +(53) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#28, d_year#29, d_moy#30] 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 -(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)) +(54) CometFilter +Input [3]: [d_date_sk#28, d_year#29, d_moy#30] +Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2001)) AND (d_moy#30 = 2)) AND isnotnull(d_date_sk#28)) -(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] +(55) CometProject +Input [3]: [d_date_sk#28, d_year#29, d_moy#30] +Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] -(57) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#30, d_moy#32] +(56) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#28, d_moy#30] -(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] +(57) BroadcastExchange +Input [2]: [d_date_sk#28, d_moy#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/simplified.txt index 0c709e4f7..a4d1753f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/simplified.txt @@ -1,75 +1,65 @@ -WholeStageCodegen (5) - Sort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] - CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [i_item_sk] #4 - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [d_date_sk,d_moy] #6 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - 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] #8 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] - CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 #9 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - ReusedExchange [i_item_sk] #4 - ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - CometBroadcastExchange [d_date_sk,d_moy] #10 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] + CometColumnarExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 + CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] + CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] + CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] + CometHashAggregate [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,w_warehouse_name,n,avg,m2,sum,count,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] + CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] + CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] + CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [i_item_sk] #4 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange [d_date_sk,d_moy] #6 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov] #7 + CometProject [mean,stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] + CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] + CometHashAggregate [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,w_warehouse_name,n,avg,m2,sum,count,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] + CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] + CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] + CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 #9 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + ReusedExchange [i_item_sk] #4 + ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 + CometBroadcastExchange [d_date_sk,d_moy] #10 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + 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-spark3_5/q39b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/explain.txt index 85590bf81..d95f2c4cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/explain.txt @@ -1,52 +1,51 @@ == Physical Plan == -* 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) +* ColumnarToRow (47) ++- CometSort (46) + +- CometColumnarExchange (45) + +- CometBroadcastHashJoin (44) + :- CometProject (23) + : +- CometFilter (22) + : +- CometHashAggregate (21) + : +- CometColumnarExchange (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) + +- CometBroadcastExchange (43) + +- CometProject (42) + +- CometFilter (41) + +- CometHashAggregate (40) + +- CometColumnarExchange (39) + +- CometHashAggregate (38) + +- CometProject (37) + +- CometBroadcastHashJoin (36) + :- CometProject (31) + : +- CometBroadcastHashJoin (30) + : :- CometProject (28) + : : +- CometBroadcastHashJoin (27) + : : :- CometFilter (25) + : : : +- CometScan parquet spark_catalog.default.inventory (24) + : : +- ReusedExchange (26) + : +- ReusedExchange (29) + +- CometBroadcastExchange (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan parquet spark_catalog.default.date_dim (32) (1) Scan parquet spark_catalog.default.inventory @@ -142,197 +141,189 @@ Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_n 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)] -(20) ColumnarToRow [codegen id : 1] +(20) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(22) HashAggregate [codegen id : 4] +(21) CometHashAggregate 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))#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] -(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) +(22) CometFilter +Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#17, mean#18] +Condition : (CASE WHEN (mean#18 = 0.0) THEN false ELSE ((stdev#17 / mean#18) > 1.0) END AND CASE WHEN (mean#18 = 0.0) THEN false ELSE ((stdev#17 / mean#18) > 1.5) END) -(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] +(23) CometProject +Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#17, mean#18] +Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19], [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, CASE WHEN (mean#18 = 0.0) THEN null ELSE (stdev#17 / mean#18) END AS cov#19] -(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] +(24) Scan parquet spark_catalog.default.inventory +Output [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#25), dynamicpruningexpression(inv_date_sk#25 IN dynamicpruning#26)] +PartitionFilters: [isnotnull(inv_date_sk#23), dynamicpruningexpression(inv_date_sk#23 IN dynamicpruning#24)] PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(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)) +(25) CometFilter +Input [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] +Condition : (isnotnull(inv_item_sk#20) AND isnotnull(inv_warehouse_sk#21)) -(27) ReusedExchange [Reuses operator id: 5] -Output [1]: [i_item_sk#27] +(26) ReusedExchange [Reuses operator id: 5] +Output [1]: [i_item_sk#25] -(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 +(27) CometBroadcastHashJoin +Left output [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] +Right output [1]: [i_item_sk#25] +Arguments: [inv_item_sk#20], [i_item_sk#25], Inner, BuildRight -(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] +(28) CometProject +Input [5]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] +Arguments: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25], [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] -(30) ReusedExchange [Reuses operator id: 10] -Output [2]: [w_warehouse_sk#28, w_warehouse_name#29] +(29) ReusedExchange [Reuses operator id: 10] +Output [2]: [w_warehouse_sk#26, w_warehouse_name#27] -(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 +(30) CometBroadcastHashJoin +Left output [4]: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] +Right output [2]: [w_warehouse_sk#26, w_warehouse_name#27] +Arguments: [inv_warehouse_sk#21], [w_warehouse_sk#26], Inner, BuildRight -(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] +(31) CometProject +Input [6]: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] +Arguments: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27], [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] -(33) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#30, d_year#31, d_moy#32] +(32) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#28, d_year#29, d_moy#30] 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 -(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)) +(33) CometFilter +Input [3]: [d_date_sk#28, d_year#29, d_moy#30] +Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2001)) AND (d_moy#30 = 2)) AND isnotnull(d_date_sk#28)) -(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] +(34) CometProject +Input [3]: [d_date_sk#28, d_year#29, d_moy#30] +Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] -(36) CometBroadcastExchange -Input [2]: [d_date_sk#30, d_moy#32] -Arguments: [d_date_sk#30, d_moy#32] +(35) CometBroadcastExchange +Input [2]: [d_date_sk#28, d_moy#30] +Arguments: [d_date_sk#28, d_moy#30] -(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 +(36) CometBroadcastHashJoin +Left output [5]: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] +Right output [2]: [d_date_sk#28, d_moy#30] +Arguments: [inv_date_sk#23], [d_date_sk#28], 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] +(37) CometProject +Input [7]: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_date_sk#28, d_moy#30] +Arguments: [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30], [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30] -(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)] +(38) CometHashAggregate +Input [5]: [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30] +Keys [4]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#22 as double)), partial_avg(inv_quantity_on_hand#22)] -(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] +(39) CometColumnarExchange +Input [9]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, n#31, avg#32, m2#33, sum#34, count#35] +Arguments: hashpartitioning(w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(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] +(40) CometHashAggregate +Input [9]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, n#31, avg#32, m2#33, sum#34, count#35] +Keys [4]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#22 as double)), avg(inv_quantity_on_hand#22)] -(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] +(41) CometFilter +Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, stdev#17, mean#18] +Condition : CASE WHEN (mean#18 = 0.0) THEN false ELSE ((stdev#17 / mean#18) > 1.0) END -(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 +(42) CometProject +Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, stdev#17, mean#18] +Arguments: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37], [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#18 AS mean#36, CASE WHEN (mean#18 = 0.0) THEN null ELSE (stdev#17 / mean#18) END AS cov#37] -(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] +(43) CometBroadcastExchange +Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] +Arguments: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -(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] +(44) CometBroadcastHashJoin +Left output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19] +Right output [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] +Arguments: [i_item_sk#6, w_warehouse_sk#7], [i_item_sk#25, w_warehouse_sk#26], Inner, BuildRight -(46) BroadcastHashJoin [codegen id : 4] -Left keys [2]: [i_item_sk#6, w_warehouse_sk#7] -Right keys [2]: [i_item_sk#27, w_warehouse_sk#28] -Join type: Inner -Join condition: None +(45) CometColumnarExchange +Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] +Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(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] +(46) CometSort +Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] +Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST] -(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 +(47) ColumnarToRow [codegen id : 1] +Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (53) -+- * ColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometScan parquet spark_catalog.default.date_dim (49) +BroadcastExchange (52) ++- * ColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) -(49) Scan parquet spark_catalog.default.date_dim +(48) 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 -(50) CometFilter +(49) 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)) -(51) CometProject +(50) 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] -(52) ColumnarToRow [codegen id : 1] +(51) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_moy#11] -(53) BroadcastExchange +(52) BroadcastExchange Input [2]: [d_date_sk#9, d_moy#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] -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) +Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 +BroadcastExchange (57) ++- * ColumnarToRow (56) + +- CometProject (55) + +- CometFilter (54) + +- CometScan parquet spark_catalog.default.date_dim (53) -(54) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#30, d_year#31, d_moy#32] +(53) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#28, d_year#29, d_moy#30] 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 -(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)) +(54) CometFilter +Input [3]: [d_date_sk#28, d_year#29, d_moy#30] +Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2001)) AND (d_moy#30 = 2)) AND isnotnull(d_date_sk#28)) -(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] +(55) CometProject +Input [3]: [d_date_sk#28, d_year#29, d_moy#30] +Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] -(57) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#30, d_moy#32] +(56) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#28, d_moy#30] -(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] +(57) BroadcastExchange +Input [2]: [d_date_sk#28, d_moy#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/simplified.txt index 0c709e4f7..a4d1753f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/simplified.txt @@ -1,75 +1,65 @@ -WholeStageCodegen (5) - Sort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] - CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [i_item_sk] #4 - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [d_date_sk,d_moy] #6 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - 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] #8 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] - CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 #9 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - ReusedExchange [i_item_sk] #4 - ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - CometBroadcastExchange [d_date_sk,d_moy] #10 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] + CometColumnarExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 + CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] + CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] + CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] + CometHashAggregate [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,w_warehouse_name,n,avg,m2,sum,count,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] + CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] + CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] + CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [i_item_sk] #4 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange [d_date_sk,d_moy] #6 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov] #7 + CometProject [mean,stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] + CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] + CometHashAggregate [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,w_warehouse_name,n,avg,m2,sum,count,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] + CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] + CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] + CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 #9 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + ReusedExchange [i_item_sk] #4 + ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 + CometBroadcastExchange [d_date_sk,d_moy] #10 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + 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-spark3_5/q4/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/explain.txt index 4aea5922b..960cb130c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/explain.txt @@ -1,110 +1,105 @@ == Physical Plan == -TakeOrderedAndProject (106) -+- * Project (105) - +- * BroadcastHashJoin Inner BuildRight (104) - :- * Project (88) - : +- * BroadcastHashJoin Inner BuildRight (87) - : :- * Project (70) - : : +- * BroadcastHashJoin Inner BuildRight (69) - : : :- * 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) - : : +- * 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) +* ColumnarToRow (101) ++- CometTakeOrderedAndProject (100) + +- CometProject (99) + +- CometBroadcastHashJoin (98) + :- CometProject (83) + : +- CometBroadcastHashJoin (82) + : :- CometProject (66) + : : +- CometBroadcastHashJoin (65) + : : :- CometProject (50) + : : : +- CometBroadcastHashJoin (49) + : : : :- CometBroadcastHashJoin (33) + : : : : :- CometFilter (16) + : : : : : +- CometHashAggregate (15) + : : : : : +- CometColumnarExchange (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) + : : : : +- CometBroadcastExchange (32) + : : : : +- CometHashAggregate (31) + : : : : +- CometColumnarExchange (30) + : : : : +- CometHashAggregate (29) + : : : : +- CometProject (28) + : : : : +- CometBroadcastHashJoin (27) + : : : : :- CometProject (23) + : : : : : +- CometBroadcastHashJoin (22) + : : : : : :- CometFilter (18) + : : : : : : +- CometScan parquet spark_catalog.default.customer (17) + : : : : : +- CometBroadcastExchange (21) + : : : : : +- CometFilter (20) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (19) + : : : : +- CometBroadcastExchange (26) + : : : : +- CometFilter (25) + : : : : +- CometScan parquet spark_catalog.default.date_dim (24) + : : : +- CometBroadcastExchange (48) + : : : +- CometFilter (47) + : : : +- CometHashAggregate (46) + : : : +- CometColumnarExchange (45) + : : : +- CometHashAggregate (44) + : : : +- CometProject (43) + : : : +- CometBroadcastHashJoin (42) + : : : :- CometProject (40) + : : : : +- CometBroadcastHashJoin (39) + : : : : :- CometFilter (35) + : : : : : +- CometScan parquet spark_catalog.default.customer (34) + : : : : +- CometBroadcastExchange (38) + : : : : +- CometFilter (37) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (36) + : : : +- ReusedExchange (41) + : : +- CometBroadcastExchange (64) + : : +- CometHashAggregate (63) + : : +- CometColumnarExchange (62) + : : +- CometHashAggregate (61) + : : +- CometProject (60) + : : +- CometBroadcastHashJoin (59) + : : :- CometProject (57) + : : : +- CometBroadcastHashJoin (56) + : : : :- CometFilter (52) + : : : : +- CometScan parquet spark_catalog.default.customer (51) + : : : +- CometBroadcastExchange (55) + : : : +- CometFilter (54) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (53) + : : +- ReusedExchange (58) + : +- CometBroadcastExchange (81) + : +- CometFilter (80) + : +- CometHashAggregate (79) + : +- CometColumnarExchange (78) + : +- CometHashAggregate (77) + : +- CometProject (76) + : +- CometBroadcastHashJoin (75) + : :- CometProject (73) + : : +- CometBroadcastHashJoin (72) + : : :- CometFilter (68) + : : : +- CometScan parquet spark_catalog.default.customer (67) + : : +- CometBroadcastExchange (71) + : : +- CometFilter (70) + : : +- CometScan parquet spark_catalog.default.web_sales (69) + : +- ReusedExchange (74) + +- CometBroadcastExchange (97) + +- CometHashAggregate (96) + +- CometColumnarExchange (95) + +- CometHashAggregate (94) + +- CometProject (93) + +- CometBroadcastHashJoin (92) + :- CometProject (90) + : +- CometBroadcastHashJoin (89) + : :- CometFilter (85) + : : +- CometScan parquet spark_catalog.default.customer (84) + : +- CometBroadcastExchange (88) + : +- CometFilter (87) + : +- CometScan parquet spark_catalog.default.web_sales (86) + +- ReusedExchange (91) (1) Scan parquet spark_catalog.default.customer @@ -172,507 +167,475 @@ Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_fl 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))] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(15) 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, 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))#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] -(17) Filter [codegen id : 12] -Input [2]: [customer_id#21, year_total#22] -Condition : (isnotnull(year_total#22) AND (year_total#22 > 0.000000)) +(16) CometFilter +Input [2]: [customer_id#20, year_total#21] +Condition : (isnotnull(year_total#21) AND (year_total#21 > 0.000000)) -(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] +(17) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#22, c_customer_id#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_country#27, c_login#28, c_email_address#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(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)) +(18) CometFilter +Input [8]: [c_customer_sk#22, c_customer_id#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_country#27, c_login#28, c_email_address#29] +Condition : (isnotnull(c_customer_sk#22) AND isnotnull(c_customer_id#23)) -(20) Scan parquet spark_catalog.default.store_sales -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] +(19) Scan parquet spark_catalog.default.store_sales +Output [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#36), dynamicpruningexpression(ss_sold_date_sk#36 IN dynamicpruning#37)] +PartitionFilters: [isnotnull(ss_sold_date_sk#35), dynamicpruningexpression(ss_sold_date_sk#35 IN dynamicpruning#36)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(21) CometFilter -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) +(20) CometFilter +Input [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Condition : isnotnull(ss_customer_sk#30) -(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] +(21) CometBroadcastExchange +Input [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Arguments: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35] -(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 +(22) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#22, c_customer_id#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_country#27, c_login#28, c_email_address#29] +Right output [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Arguments: [c_customer_sk#22], [ss_customer_sk#30], Inner, BuildRight -(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] +(23) CometProject +Input [14]: [c_customer_sk#22, c_customer_id#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_country#27, c_login#28, c_email_address#29, ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Arguments: [c_customer_id#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_country#27, c_login#28, c_email_address#29, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35], [c_customer_id#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_country#27, c_login#28, c_email_address#29, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35] -(25) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#38, d_year#39] +(24) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#37, d_year#38] 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 -(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 - -(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] +(25) CometFilter +Input [2]: [d_date_sk#37, d_year#38] +Condition : ((isnotnull(d_year#38) AND (d_year#38 = 2002)) AND isnotnull(d_date_sk#37)) + +(26) CometBroadcastExchange +Input [2]: [d_date_sk#37, d_year#38] +Arguments: [d_date_sk#37, d_year#38] + +(27) CometBroadcastHashJoin +Left output [12]: [c_customer_id#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_country#27, c_login#28, c_email_address#29, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Right output [2]: [d_date_sk#37, d_year#38] +Arguments: [ss_sold_date_sk#35], [d_date_sk#37], Inner, BuildRight + +(28) CometProject +Input [14]: [c_customer_id#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_country#27, c_login#28, c_email_address#29, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35, d_date_sk#37, d_year#38] +Arguments: [c_customer_id#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_country#27, c_login#28, c_email_address#29, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38], [c_customer_id#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_country#27, c_login#28, c_email_address#29, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38] + +(29) CometHashAggregate +Input [12]: [c_customer_id#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_country#27, c_login#28, c_email_address#29, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38] +Keys [8]: [c_customer_id#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_country#27, c_login#28, c_email_address#29, d_year#38] +Functions [1]: [partial_sum(((((ss_ext_list_price#34 - ss_ext_wholesale_cost#33) - ss_ext_discount_amt#31) + ss_ext_sales_price#32) / 2))] + +(30) CometColumnarExchange +Input [10]: [c_customer_id#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_country#27, c_login#28, c_email_address#29, d_year#38, sum#39, isEmpty#40] +Arguments: hashpartitioning(c_customer_id#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_country#27, c_login#28, c_email_address#29, d_year#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(31) CometHashAggregate +Input [10]: [c_customer_id#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_country#27, c_login#28, c_email_address#29, d_year#38, sum#39, isEmpty#40] +Keys [8]: [c_customer_id#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_country#27, c_login#28, c_email_address#29, d_year#38] +Functions [1]: [sum(((((ss_ext_list_price#34 - ss_ext_wholesale_cost#33) - ss_ext_discount_amt#31) + ss_ext_sales_price#32) / 2))] + +(32) CometBroadcastExchange +Input [8]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47, year_total#48] +Arguments: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47, year_total#48] + +(33) CometBroadcastHashJoin +Left output [2]: [customer_id#20, year_total#21] +Right output [8]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47, year_total#48] +Arguments: [customer_id#20], [customer_id#41], Inner, BuildRight + +(34) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#49, c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(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)) +(35) CometFilter +Input [8]: [c_customer_sk#49, c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56] +Condition : (isnotnull(c_customer_sk#49) AND isnotnull(c_customer_id#50)) -(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] +(36) Scan parquet spark_catalog.default.catalog_sales +Output [6]: [cs_bill_customer_sk#57, cs_ext_discount_amt#58, cs_ext_sales_price#59, cs_ext_wholesale_cost#60, cs_ext_list_price#61, cs_sold_date_sk#62] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#63), dynamicpruningexpression(cs_sold_date_sk#63 IN dynamicpruning#64)] +PartitionFilters: [isnotnull(cs_sold_date_sk#62), dynamicpruningexpression(cs_sold_date_sk#62 IN dynamicpruning#63)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(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) +(37) CometFilter +Input [6]: [cs_bill_customer_sk#57, cs_ext_discount_amt#58, cs_ext_sales_price#59, cs_ext_wholesale_cost#60, cs_ext_list_price#61, cs_sold_date_sk#62] +Condition : isnotnull(cs_bill_customer_sk#57) + +(38) CometBroadcastExchange +Input [6]: [cs_bill_customer_sk#57, cs_ext_discount_amt#58, cs_ext_sales_price#59, cs_ext_wholesale_cost#60, cs_ext_list_price#61, cs_sold_date_sk#62] +Arguments: [cs_bill_customer_sk#57, cs_ext_discount_amt#58, cs_ext_sales_price#59, cs_ext_wholesale_cost#60, cs_ext_list_price#61, cs_sold_date_sk#62] + +(39) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#49, c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56] +Right output [6]: [cs_bill_customer_sk#57, cs_ext_discount_amt#58, cs_ext_sales_price#59, cs_ext_wholesale_cost#60, cs_ext_list_price#61, cs_sold_date_sk#62] +Arguments: [c_customer_sk#49], [cs_bill_customer_sk#57], Inner, BuildRight -(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) CometProject +Input [14]: [c_customer_sk#49, c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, cs_bill_customer_sk#57, cs_ext_discount_amt#58, cs_ext_sales_price#59, cs_ext_wholesale_cost#60, cs_ext_list_price#61, cs_sold_date_sk#62] +Arguments: [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, cs_ext_discount_amt#58, cs_ext_sales_price#59, cs_ext_wholesale_cost#60, cs_ext_list_price#61, cs_sold_date_sk#62], [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, cs_ext_discount_amt#58, cs_ext_sales_price#59, cs_ext_wholesale_cost#60, cs_ext_list_price#61, cs_sold_date_sk#62] -(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) ReusedExchange [Reuses operator id: 10] +Output [2]: [d_date_sk#64, d_year#65] -(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) CometBroadcastHashJoin +Left output [12]: [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, cs_ext_discount_amt#58, cs_ext_sales_price#59, cs_ext_wholesale_cost#60, cs_ext_list_price#61, cs_sold_date_sk#62] +Right output [2]: [d_date_sk#64, d_year#65] +Arguments: [cs_sold_date_sk#62], [d_date_sk#64], Inner, BuildRight -(43) ReusedExchange [Reuses operator id: 10] -Output [2]: [d_date_sk#65, d_year#66] +(43) CometProject +Input [14]: [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, cs_ext_discount_amt#58, cs_ext_sales_price#59, cs_ext_wholesale_cost#60, cs_ext_list_price#61, cs_sold_date_sk#62, d_date_sk#64, d_year#65] +Arguments: [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, cs_ext_discount_amt#58, cs_ext_sales_price#59, cs_ext_wholesale_cost#60, cs_ext_list_price#61, d_year#65], [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, cs_ext_discount_amt#58, cs_ext_sales_price#59, cs_ext_wholesale_cost#60, cs_ext_list_price#61, d_year#65] -(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) CometHashAggregate +Input [12]: [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, cs_ext_discount_amt#58, cs_ext_sales_price#59, cs_ext_wholesale_cost#60, cs_ext_list_price#61, d_year#65] +Keys [8]: [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, d_year#65] +Functions [1]: [partial_sum(((((cs_ext_list_price#61 - cs_ext_wholesale_cost#60) - cs_ext_discount_amt#58) + cs_ext_sales_price#59) / 2))] -(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] +(45) CometColumnarExchange +Input [10]: [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, d_year#65, sum#66, isEmpty#67] +Arguments: hashpartitioning(c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, d_year#65, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (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))] - -(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 - -(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] - -(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] +Input [10]: [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, d_year#65, sum#66, isEmpty#67] +Keys [8]: [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, d_year#65] +Functions [1]: [sum(((((cs_ext_list_price#61 - cs_ext_wholesale_cost#60) - cs_ext_discount_amt#58) + cs_ext_sales_price#59) / 2))] + +(47) CometFilter +Input [2]: [customer_id#68, year_total#69] +Condition : (isnotnull(year_total#69) AND (year_total#69 > 0.000000)) + +(48) CometBroadcastExchange +Input [2]: [customer_id#68, year_total#69] +Arguments: [customer_id#68, year_total#69] + +(49) CometBroadcastHashJoin +Left output [10]: [customer_id#20, year_total#21, customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47, year_total#48] +Right output [2]: [customer_id#68, year_total#69] +Arguments: [customer_id#20], [customer_id#68], Inner, BuildRight + +(50) CometProject +Input [12]: [customer_id#20, year_total#21, customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47, year_total#48, customer_id#68, year_total#69] +Arguments: [customer_id#20, year_total#21, customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47, year_total#48, year_total#69], [customer_id#20, year_total#21, customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47, year_total#48, year_total#69] + +(51) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] 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#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)) +(52) CometFilter +Input [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] +Condition : (isnotnull(c_customer_sk#70) AND isnotnull(c_customer_id#71)) -(56) Scan parquet spark_catalog.default.catalog_sales -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] +(53) Scan parquet spark_catalog.default.catalog_sales +Output [6]: [cs_bill_customer_sk#78, cs_ext_discount_amt#79, cs_ext_sales_price#80, cs_ext_wholesale_cost#81, cs_ext_list_price#82, cs_sold_date_sk#83] 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#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#84)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(57) CometFilter -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) +(54) CometFilter +Input [6]: [cs_bill_customer_sk#78, cs_ext_discount_amt#79, cs_ext_sales_price#80, cs_ext_wholesale_cost#81, cs_ext_list_price#82, cs_sold_date_sk#83] +Condition : isnotnull(cs_bill_customer_sk#78) + +(55) CometBroadcastExchange +Input [6]: [cs_bill_customer_sk#78, cs_ext_discount_amt#79, cs_ext_sales_price#80, cs_ext_wholesale_cost#81, cs_ext_list_price#82, cs_sold_date_sk#83] +Arguments: [cs_bill_customer_sk#78, cs_ext_discount_amt#79, cs_ext_sales_price#80, cs_ext_wholesale_cost#81, cs_ext_list_price#82, cs_sold_date_sk#83] + +(56) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] +Right output [6]: [cs_bill_customer_sk#78, cs_ext_discount_amt#79, cs_ext_sales_price#80, cs_ext_wholesale_cost#81, cs_ext_list_price#82, cs_sold_date_sk#83] +Arguments: [c_customer_sk#70], [cs_bill_customer_sk#78], Inner, BuildRight -(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] +(57) CometProject +Input [14]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, cs_bill_customer_sk#78, cs_ext_discount_amt#79, cs_ext_sales_price#80, cs_ext_wholesale_cost#81, cs_ext_list_price#82, cs_sold_date_sk#83] +Arguments: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, cs_ext_discount_amt#79, cs_ext_sales_price#80, cs_ext_wholesale_cost#81, cs_ext_list_price#82, cs_sold_date_sk#83], [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, cs_ext_discount_amt#79, cs_ext_sales_price#80, cs_ext_wholesale_cost#81, cs_ext_list_price#82, cs_sold_date_sk#83] + +(58) ReusedExchange [Reuses operator id: 26] +Output [2]: [d_date_sk#85, d_year#86] (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 +Left output [12]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, cs_ext_discount_amt#79, cs_ext_sales_price#80, cs_ext_wholesale_cost#81, cs_ext_list_price#82, cs_sold_date_sk#83] +Right output [2]: [d_date_sk#85, d_year#86] +Arguments: [cs_sold_date_sk#83], [d_date_sk#85], Inner, BuildRight (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) ReusedExchange [Reuses operator id: 27] -Output [2]: [d_date_sk#87, d_year#88] - -(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) 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) 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) 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#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 : 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#91, year_total#92] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] - -(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#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 : 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#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] +Input [14]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, cs_ext_discount_amt#79, cs_ext_sales_price#80, cs_ext_wholesale_cost#81, cs_ext_list_price#82, cs_sold_date_sk#83, d_date_sk#85, d_year#86] +Arguments: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, cs_ext_discount_amt#79, cs_ext_sales_price#80, cs_ext_wholesale_cost#81, cs_ext_list_price#82, d_year#86], [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, cs_ext_discount_amt#79, cs_ext_sales_price#80, cs_ext_wholesale_cost#81, cs_ext_list_price#82, d_year#86] + +(61) CometHashAggregate +Input [12]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, cs_ext_discount_amt#79, cs_ext_sales_price#80, cs_ext_wholesale_cost#81, cs_ext_list_price#82, d_year#86] +Keys [8]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, d_year#86] +Functions [1]: [partial_sum(((((cs_ext_list_price#82 - cs_ext_wholesale_cost#81) - cs_ext_discount_amt#79) + cs_ext_sales_price#80) / 2))] + +(62) CometColumnarExchange +Input [10]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, d_year#86, sum#87, isEmpty#88] +Arguments: hashpartitioning(c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, d_year#86, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(63) CometHashAggregate +Input [10]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, d_year#86, sum#87, isEmpty#88] +Keys [8]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, d_year#86] +Functions [1]: [sum(((((cs_ext_list_price#82 - cs_ext_wholesale_cost#81) - cs_ext_discount_amt#79) + cs_ext_sales_price#80) / 2))] + +(64) CometBroadcastExchange +Input [2]: [customer_id#89, year_total#90] +Arguments: [customer_id#89, year_total#90] + +(65) CometBroadcastHashJoin +Left output [11]: [customer_id#20, year_total#21, customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47, year_total#48, year_total#69] +Right output [2]: [customer_id#89, year_total#90] +Arguments: [customer_id#20], [customer_id#89], Inner, (CASE WHEN (year_total#69 > 0.000000) THEN (year_total#90 / year_total#69) END > CASE WHEN (year_total#21 > 0.000000) THEN (year_total#48 / year_total#21) END), BuildRight + +(66) CometProject +Input [13]: [customer_id#20, year_total#21, customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47, year_total#48, year_total#69, customer_id#89, year_total#90] +Arguments: [customer_id#20, customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47, year_total#69, year_total#90], [customer_id#20, customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47, year_total#69, year_total#90] + +(67) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#91, c_customer_id#92, c_first_name#93, c_last_name#94, c_preferred_cust_flag#95, c_birth_country#96, c_login#97, c_email_address#98] 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#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)) +(68) CometFilter +Input [8]: [c_customer_sk#91, c_customer_id#92, c_first_name#93, c_last_name#94, c_preferred_cust_flag#95, c_birth_country#96, c_login#97, c_email_address#98] +Condition : (isnotnull(c_customer_sk#91) AND isnotnull(c_customer_id#92)) -(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] +(69) Scan parquet spark_catalog.default.web_sales +Output [6]: [ws_bill_customer_sk#99, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, ws_sold_date_sk#104] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#106), dynamicpruningexpression(ws_sold_date_sk#106 IN dynamicpruning#107)] +PartitionFilters: [isnotnull(ws_sold_date_sk#104), dynamicpruningexpression(ws_sold_date_sk#104 IN dynamicpruning#105)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(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 - -(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] - -(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] +(70) CometFilter +Input [6]: [ws_bill_customer_sk#99, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, ws_sold_date_sk#104] +Condition : isnotnull(ws_bill_customer_sk#99) + +(71) CometBroadcastExchange +Input [6]: [ws_bill_customer_sk#99, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, ws_sold_date_sk#104] +Arguments: [ws_bill_customer_sk#99, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, ws_sold_date_sk#104] + +(72) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#91, c_customer_id#92, c_first_name#93, c_last_name#94, c_preferred_cust_flag#95, c_birth_country#96, c_login#97, c_email_address#98] +Right output [6]: [ws_bill_customer_sk#99, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, ws_sold_date_sk#104] +Arguments: [c_customer_sk#91], [ws_bill_customer_sk#99], Inner, BuildRight + +(73) CometProject +Input [14]: [c_customer_sk#91, c_customer_id#92, c_first_name#93, c_last_name#94, c_preferred_cust_flag#95, c_birth_country#96, c_login#97, c_email_address#98, ws_bill_customer_sk#99, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, ws_sold_date_sk#104] +Arguments: [c_customer_id#92, c_first_name#93, c_last_name#94, c_preferred_cust_flag#95, c_birth_country#96, c_login#97, c_email_address#98, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, ws_sold_date_sk#104], [c_customer_id#92, c_first_name#93, c_last_name#94, c_preferred_cust_flag#95, c_birth_country#96, c_login#97, c_email_address#98, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, ws_sold_date_sk#104] + +(74) ReusedExchange [Reuses operator id: 10] +Output [2]: [d_date_sk#106, d_year#107] + +(75) CometBroadcastHashJoin +Left output [12]: [c_customer_id#92, c_first_name#93, c_last_name#94, c_preferred_cust_flag#95, c_birth_country#96, c_login#97, c_email_address#98, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, ws_sold_date_sk#104] +Right output [2]: [d_date_sk#106, d_year#107] +Arguments: [ws_sold_date_sk#104], [d_date_sk#106], Inner, BuildRight + +(76) CometProject +Input [14]: [c_customer_id#92, c_first_name#93, c_last_name#94, c_preferred_cust_flag#95, c_birth_country#96, c_login#97, c_email_address#98, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, ws_sold_date_sk#104, d_date_sk#106, d_year#107] +Arguments: [c_customer_id#92, c_first_name#93, c_last_name#94, c_preferred_cust_flag#95, c_birth_country#96, c_login#97, c_email_address#98, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, d_year#107], [c_customer_id#92, c_first_name#93, c_last_name#94, c_preferred_cust_flag#95, c_birth_country#96, c_login#97, c_email_address#98, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, d_year#107] + +(77) CometHashAggregate +Input [12]: [c_customer_id#92, c_first_name#93, c_last_name#94, c_preferred_cust_flag#95, c_birth_country#96, c_login#97, c_email_address#98, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, d_year#107] +Keys [8]: [c_customer_id#92, c_first_name#93, c_last_name#94, c_preferred_cust_flag#95, c_birth_country#96, c_login#97, c_email_address#98, d_year#107] +Functions [1]: [partial_sum(((((ws_ext_list_price#103 - ws_ext_wholesale_cost#102) - ws_ext_discount_amt#100) + ws_ext_sales_price#101) / 2))] + +(78) CometColumnarExchange +Input [10]: [c_customer_id#92, c_first_name#93, c_last_name#94, c_preferred_cust_flag#95, c_birth_country#96, c_login#97, c_email_address#98, d_year#107, sum#108, isEmpty#109] +Arguments: hashpartitioning(c_customer_id#92, c_first_name#93, c_last_name#94, c_preferred_cust_flag#95, c_birth_country#96, c_login#97, c_email_address#98, d_year#107, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(79) CometHashAggregate +Input [10]: [c_customer_id#92, c_first_name#93, c_last_name#94, c_preferred_cust_flag#95, c_birth_country#96, c_login#97, c_email_address#98, d_year#107, sum#108, isEmpty#109] +Keys [8]: [c_customer_id#92, c_first_name#93, c_last_name#94, c_preferred_cust_flag#95, c_birth_country#96, c_login#97, c_email_address#98, d_year#107] +Functions [1]: [sum(((((ws_ext_list_price#103 - ws_ext_wholesale_cost#102) - ws_ext_discount_amt#100) + ws_ext_sales_price#101) / 2))] + +(80) CometFilter +Input [2]: [customer_id#110, year_total#111] +Condition : (isnotnull(year_total#111) AND (year_total#111 > 0.000000)) + +(81) CometBroadcastExchange +Input [2]: [customer_id#110, year_total#111] +Arguments: [customer_id#110, year_total#111] + +(82) CometBroadcastHashJoin +Left output [10]: [customer_id#20, customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47, year_total#69, year_total#90] +Right output [2]: [customer_id#110, year_total#111] +Arguments: [customer_id#20], [customer_id#110], Inner, BuildRight + +(83) CometProject +Input [12]: [customer_id#20, customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47, year_total#69, year_total#90, customer_id#110, year_total#111] +Arguments: [customer_id#20, customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47, year_total#69, year_total#90, year_total#111], [customer_id#20, customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47, year_total#69, year_total#90, year_total#111] + +(84) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#112, c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(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)) +(85) CometFilter +Input [8]: [c_customer_sk#112, c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119] +Condition : (isnotnull(c_customer_sk#112) AND isnotnull(c_customer_id#113)) -(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] +(86) Scan parquet spark_catalog.default.web_sales +Output [6]: [ws_bill_customer_sk#120, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, ws_sold_date_sk#125] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#128), dynamicpruningexpression(ws_sold_date_sk#128 IN dynamicpruning#129)] +PartitionFilters: [isnotnull(ws_sold_date_sk#125), dynamicpruningexpression(ws_sold_date_sk#125 IN dynamicpruning#126)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(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) +(87) CometFilter +Input [6]: [ws_bill_customer_sk#120, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, ws_sold_date_sk#125] +Condition : isnotnull(ws_bill_customer_sk#120) -(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] +(88) CometBroadcastExchange +Input [6]: [ws_bill_customer_sk#120, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, ws_sold_date_sk#125] +Arguments: [ws_bill_customer_sk#120, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, ws_sold_date_sk#125] -(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 +(89) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#112, c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119] +Right output [6]: [ws_bill_customer_sk#120, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, ws_sold_date_sk#125] +Arguments: [c_customer_sk#112], [ws_bill_customer_sk#120], Inner, BuildRight -(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] +(90) CometProject +Input [14]: [c_customer_sk#112, c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, ws_bill_customer_sk#120, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, ws_sold_date_sk#125] +Arguments: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, ws_sold_date_sk#125], [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, ws_sold_date_sk#125] -(96) ReusedExchange [Reuses operator id: 27] -Output [2]: [d_date_sk#130, d_year#131] +(91) ReusedExchange [Reuses operator id: 26] +Output [2]: [d_date_sk#127, d_year#128] -(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 +(92) CometBroadcastHashJoin +Left output [12]: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, ws_sold_date_sk#125] +Right output [2]: [d_date_sk#127, d_year#128] +Arguments: [ws_sold_date_sk#125], [d_date_sk#127], Inner, BuildRight -(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] +(93) CometProject +Input [14]: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, ws_sold_date_sk#125, d_date_sk#127, d_year#128] +Arguments: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, d_year#128], [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, d_year#128] -(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))] +(94) CometHashAggregate +Input [12]: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, d_year#128] +Keys [8]: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, d_year#128] +Functions [1]: [partial_sum(((((ws_ext_list_price#124 - ws_ext_wholesale_cost#123) - ws_ext_discount_amt#121) + ws_ext_sales_price#122) / 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] +(95) CometColumnarExchange +Input [10]: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, d_year#128, sum#129, isEmpty#130] +Arguments: hashpartitioning(c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, d_year#128, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(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] +(96) CometHashAggregate +Input [10]: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, d_year#128, sum#129, isEmpty#130] +Keys [8]: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, d_year#128] +Functions [1]: [sum(((((ws_ext_list_price#124 - ws_ext_wholesale_cost#123) - ws_ext_discount_amt#121) + ws_ext_sales_price#122) / 2))] -(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] +(97) CometBroadcastExchange +Input [2]: [customer_id#131, year_total#132] +Arguments: [customer_id#131, year_total#132] -(103) BroadcastExchange -Input [2]: [customer_id#134, year_total#135] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] +(98) CometBroadcastHashJoin +Left output [11]: [customer_id#20, customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47, year_total#69, year_total#90, year_total#111] +Right output [2]: [customer_id#131, year_total#132] +Arguments: [customer_id#20], [customer_id#131], Inner, (CASE WHEN (year_total#69 > 0.000000) THEN (year_total#90 / year_total#69) END > CASE WHEN (year_total#111 > 0.000000) THEN (year_total#132 / year_total#111) END), BuildRight -(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#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) +(99) CometProject +Input [13]: [customer_id#20, customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47, year_total#69, year_total#90, year_total#111, customer_id#131, year_total#132] +Arguments: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47], [customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47] -(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] +(100) CometTakeOrderedAndProject +Input [7]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#41 ASC NULLS FIRST,customer_first_name#42 ASC NULLS FIRST,customer_last_name#43 ASC NULLS FIRST,customer_preferred_cust_flag#44 ASC NULLS FIRST,customer_birth_country#45 ASC NULLS FIRST,customer_login#46 ASC NULLS FIRST,customer_email_address#47 ASC NULLS FIRST], output=[customer_id#41,customer_first_name#42,customer_last_name#43,customer_preferred_cust_flag#44,customer_birth_country#45,customer_login#46,customer_email_address#47]), [customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47], 100, [customer_id#41 ASC NULLS FIRST, customer_first_name#42 ASC NULLS FIRST, customer_last_name#43 ASC NULLS FIRST, customer_preferred_cust_flag#44 ASC NULLS FIRST, customer_birth_country#45 ASC NULLS FIRST, customer_login#46 ASC NULLS FIRST, customer_email_address#47 ASC NULLS FIRST], [customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47] -(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] +(101) ColumnarToRow [codegen id : 1] +Input [7]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47] ===== Subqueries ===== 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) +BroadcastExchange (105) ++- * ColumnarToRow (104) + +- CometFilter (103) + +- CometScan parquet spark_catalog.default.date_dim (102) -(107) Scan parquet spark_catalog.default.date_dim +(102) 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 -(108) CometFilter +(103) 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)) -(109) ColumnarToRow [codegen id : 1] +(104) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#16, d_year#17] -(110) BroadcastExchange +(105) BroadcastExchange Input [2]: [d_date_sk#16, d_year#17] -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=7] -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) +Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#35 IN dynamicpruning#36 +BroadcastExchange (109) ++- * ColumnarToRow (108) + +- CometFilter (107) + +- CometScan parquet spark_catalog.default.date_dim (106) -(111) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#38, d_year#39] +(106) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#37, d_year#38] 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 -(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)) +(107) CometFilter +Input [2]: [d_date_sk#37, d_year#38] +Condition : ((isnotnull(d_year#38) AND (d_year#38 = 2002)) AND isnotnull(d_date_sk#37)) -(113) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#38, d_year#39] +(108) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#37, d_year#38] -(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] +(109) BroadcastExchange +Input [2]: [d_date_sk#37, d_year#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -Subquery:3 Hosting operator id = 38 Hosting Expression = cs_sold_date_sk#63 IN dynamicpruning#15 +Subquery:3 Hosting operator id = 36 Hosting Expression = cs_sold_date_sk#62 IN dynamicpruning#15 -Subquery:4 Hosting operator id = 56 Hosting Expression = cs_sold_date_sk#85 IN dynamicpruning#37 +Subquery:4 Hosting operator id = 53 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#36 -Subquery:5 Hosting operator id = 73 Hosting Expression = ws_sold_date_sk#106 IN dynamicpruning#15 +Subquery:5 Hosting operator id = 69 Hosting Expression = ws_sold_date_sk#104 IN dynamicpruning#15 -Subquery:6 Hosting operator id = 91 Hosting Expression = ws_sold_date_sk#128 IN dynamicpruning#37 +Subquery:6 Hosting operator id = 86 Hosting Expression = ws_sold_date_sk#125 IN dynamicpruning#36 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/simplified.txt index 1a19c58ff..7e6b12575 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/simplified.txt @@ -1,153 +1,121 @@ -TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] - 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] - BroadcastHashJoin [customer_id,customer_id] - 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] - 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_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id] - BroadcastHashJoin [customer_id,customer_id] - 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(((((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 (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,sum,isEmpty,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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_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_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - 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,sum,isEmpty,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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #7 - CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_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 #8 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - 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] #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,sum,isEmpty,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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #12 - CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_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 #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] #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,sum,isEmpty,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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #15 - CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_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 #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] #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,sum,isEmpty,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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] + CometProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] + CometBroadcastHashJoin [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,customer_id,year_total] + CometProject [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] + CometBroadcastHashJoin [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,customer_id,year_total] + CometProject [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] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,customer_id,year_total] + CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,customer_id,year_total] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total] + CometFilter [customer_id,year_total] + CometHashAggregate [customer_id,year_total,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))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #1 + CometHashAggregate [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,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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #18 - CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_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] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_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_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total] #5 + CometHashAggregate [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,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))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #6 + CometHashAggregate [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,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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + 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 [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_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 #8 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [customer_id,year_total] #10 + CometFilter [customer_id,year_total] + CometHashAggregate [customer_id,year_total,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))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 + CometHashAggregate [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,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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + 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 [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #12 + CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_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 #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] #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,sum,isEmpty,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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + CometBroadcastExchange [customer_id,year_total] #13 + CometHashAggregate [customer_id,year_total,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))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 + CometHashAggregate [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,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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #21 - CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_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] + CometBroadcastExchange [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #15 + CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_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 + CometBroadcastExchange [customer_id,year_total] #16 + CometFilter [customer_id,year_total] + CometHashAggregate [customer_id,year_total,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))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #17 + CometHashAggregate [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,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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #18 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_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 + CometBroadcastExchange [customer_id,year_total] #19 + CometHashAggregate [customer_id,year_total,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))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #20 + CometHashAggregate [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,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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #21 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_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-spark3_5/q40/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/explain.txt index f63b94658..00967faa4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/explain.txt @@ -1,37 +1,36 @@ == Physical Plan == -TakeOrderedAndProject (33) -+- * HashAggregate (32) - +- Exchange (31) - +- * HashAggregate (30) - +- * Project (29) - +- * BroadcastHashJoin Inner BuildRight (28) - :- * Project (26) - : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (13) - : : : +- * SortMergeJoin LeftOuter (12) - : : : :- * Sort (5) - : : : : +- Exchange (4) - : : : : +- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : +- * Sort (11) - : : : +- Exchange (10) - : : : +- * ColumnarToRow (9) - : : : +- CometProject (8) - : : : +- CometFilter (7) - : : : +- CometScan parquet spark_catalog.default.catalog_returns (6) - : : +- BroadcastExchange (17) - : : +- * ColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometScan parquet spark_catalog.default.warehouse (14) - : +- BroadcastExchange (24) - : +- * ColumnarToRow (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan parquet spark_catalog.default.item (20) - +- ReusedExchange (27) +* ColumnarToRow (32) ++- CometTakeOrderedAndProject (31) + +- CometHashAggregate (30) + +- CometColumnarExchange (29) + +- CometHashAggregate (28) + +- CometProject (27) + +- CometBroadcastHashJoin (26) + :- CometProject (22) + : +- CometBroadcastHashJoin (21) + : :- CometProject (16) + : : +- CometBroadcastHashJoin (15) + : : :- CometProject (11) + : : : +- CometSortMergeJoin (10) + : : : :- CometSort (4) + : : : : +- CometColumnarExchange (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : +- CometSort (9) + : : : +- CometColumnarExchange (8) + : : : +- CometProject (7) + : : : +- CometFilter (6) + : : : +- CometScan parquet spark_catalog.default.catalog_returns (5) + : : +- CometBroadcastExchange (14) + : : +- CometFilter (13) + : : +- CometScan parquet spark_catalog.default.warehouse (12) + : +- CometBroadcastExchange (20) + : +- CometProject (19) + : +- CometFilter (18) + : +- CometScan parquet spark_catalog.default.item (17) + +- CometBroadcastExchange (25) + +- CometFilter (24) + +- CometScan parquet spark_catalog.default.date_dim (23) (1) Scan parquet spark_catalog.default.catalog_sales @@ -46,173 +45,168 @@ ReadSchema: struct -(7) CometFilter +(6) CometFilter Input [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] Condition : (isnotnull(cr_order_number#8) AND isnotnull(cr_item_sk#7)) -(8) CometProject +(7) CometProject Input [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] Arguments: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9], [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -(9) ColumnarToRow [codegen id : 3] +(8) CometColumnarExchange Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] +Arguments: hashpartitioning(cr_order_number#8, cr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(10) Exchange +(9) CometSort Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: hashpartitioning(cr_order_number#8, cr_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9], [cr_order_number#8 ASC NULLS FIRST, cr_item_sk#7 ASC NULLS FIRST] -(11) Sort [codegen id : 4] -Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cr_order_number#8 ASC NULLS FIRST, cr_item_sk#7 ASC NULLS FIRST], false, 0 +(10) CometSortMergeJoin +Left output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] +Right output [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] +Arguments: [cs_order_number#3, cs_item_sk#2], [cr_order_number#8, cr_item_sk#7], LeftOuter -(12) SortMergeJoin [codegen id : 8] -Left keys [2]: [cs_order_number#3, cs_item_sk#2] -Right keys [2]: [cr_order_number#8, cr_item_sk#7] -Join type: LeftOuter -Join condition: None - -(13) Project [codegen id : 8] -Output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9] +(11) CometProject Input [8]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5, cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] +Arguments: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9], [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9] -(14) Scan parquet spark_catalog.default.warehouse +(12) Scan parquet spark_catalog.default.warehouse Output [2]: [w_warehouse_sk#11, w_state#12] Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] PushedFilters: [IsNotNull(w_warehouse_sk)] ReadSchema: struct -(15) CometFilter +(13) CometFilter Input [2]: [w_warehouse_sk#11, w_state#12] Condition : isnotnull(w_warehouse_sk#11) -(16) ColumnarToRow [codegen id : 5] -Input [2]: [w_warehouse_sk#11, w_state#12] - -(17) BroadcastExchange +(14) CometBroadcastExchange Input [2]: [w_warehouse_sk#11, w_state#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +Arguments: [w_warehouse_sk#11, w_state#12] -(18) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_warehouse_sk#1] -Right keys [1]: [w_warehouse_sk#11] -Join type: Inner -Join condition: None +(15) CometBroadcastHashJoin +Left output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9] +Right output [2]: [w_warehouse_sk#11, w_state#12] +Arguments: [cs_warehouse_sk#1], [w_warehouse_sk#11], Inner, BuildRight -(19) Project [codegen id : 8] -Output [5]: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12] +(16) CometProject Input [7]: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_warehouse_sk#11, w_state#12] +Arguments: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12], [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12] -(20) Scan parquet spark_catalog.default.item +(17) Scan parquet spark_catalog.default.item Output [3]: [i_item_sk#13, i_item_id#14, i_current_price#15] 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 -(21) CometFilter +(18) CometFilter Input [3]: [i_item_sk#13, i_item_id#14, i_current_price#15] Condition : (((isnotnull(i_current_price#15) AND (i_current_price#15 >= 0.99)) AND (i_current_price#15 <= 1.49)) AND isnotnull(i_item_sk#13)) -(22) CometProject +(19) CometProject Input [3]: [i_item_sk#13, i_item_id#14, i_current_price#15] Arguments: [i_item_sk#13, i_item_id#14], [i_item_sk#13, i_item_id#14] -(23) ColumnarToRow [codegen id : 6] +(20) CometBroadcastExchange Input [2]: [i_item_sk#13, i_item_id#14] +Arguments: [i_item_sk#13, i_item_id#14] -(24) BroadcastExchange -Input [2]: [i_item_sk#13, i_item_id#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(21) CometBroadcastHashJoin +Left output [5]: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12] +Right output [2]: [i_item_sk#13, i_item_id#14] +Arguments: [cs_item_sk#2], [i_item_sk#13], Inner, BuildRight -(25) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_item_sk#2] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None - -(26) Project [codegen id : 8] -Output [5]: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12, i_item_id#14] +(22) CometProject Input [7]: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12, i_item_sk#13, i_item_id#14] +Arguments: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12, i_item_id#14], [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12, i_item_id#14] -(27) ReusedExchange [Reuses operator id: 37] +(23) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#16, d_date#17] +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 + +(24) CometFilter +Input [2]: [d_date_sk#16, d_date#17] +Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 2000-02-10)) AND (d_date#17 <= 2000-04-10)) AND isnotnull(d_date_sk#16)) -(28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#16] -Join type: Inner -Join condition: None +(25) CometBroadcastExchange +Input [2]: [d_date_sk#16, d_date#17] +Arguments: [d_date_sk#16, d_date#17] -(29) Project [codegen id : 8] -Output [5]: [cs_sales_price#4, cr_refunded_cash#9, w_state#12, i_item_id#14, d_date#17] +(26) CometBroadcastHashJoin +Left output [5]: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12, i_item_id#14] +Right output [2]: [d_date_sk#16, d_date#17] +Arguments: [cs_sold_date_sk#5], [d_date_sk#16], Inner, BuildRight + +(27) CometProject Input [7]: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12, i_item_id#14, d_date_sk#16, d_date#17] +Arguments: [cs_sales_price#4, cr_refunded_cash#9, w_state#12, i_item_id#14, d_date#17], [cs_sales_price#4, cr_refunded_cash#9, w_state#12, i_item_id#14, d_date#17] -(30) HashAggregate [codegen id : 8] +(28) CometHashAggregate Input [5]: [cs_sales_price#4, cr_refunded_cash#9, w_state#12, i_item_id#14, d_date#17] Keys [2]: [w_state#12, i_item_id#14] Functions [2]: [partial_sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END), partial_sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)] -Aggregate Attributes [4]: [sum#18, isEmpty#19, sum#20, isEmpty#21] -Results [6]: [w_state#12, i_item_id#14, sum#22, isEmpty#23, sum#24, isEmpty#25] -(31) Exchange -Input [6]: [w_state#12, i_item_id#14, sum#22, isEmpty#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(w_state#12, i_item_id#14, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(29) CometColumnarExchange +Input [6]: [w_state#12, i_item_id#14, sum#18, isEmpty#19, sum#20, isEmpty#21] +Arguments: hashpartitioning(w_state#12, i_item_id#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(32) HashAggregate [codegen id : 9] -Input [6]: [w_state#12, i_item_id#14, sum#22, isEmpty#23, sum#24, isEmpty#25] +(30) CometHashAggregate +Input [6]: [w_state#12, i_item_id#14, sum#18, isEmpty#19, sum#20, isEmpty#21] Keys [2]: [w_state#12, i_item_id#14] Functions [2]: [sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END), sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)] -Aggregate Attributes [2]: [sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#26, sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#27] -Results [4]: [w_state#12, i_item_id#14, sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#26 AS sales_before#28, sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#27 AS sales_after#29] -(33) TakeOrderedAndProject -Input [4]: [w_state#12, i_item_id#14, sales_before#28, sales_after#29] -Arguments: 100, [w_state#12 ASC NULLS FIRST, i_item_id#14 ASC NULLS FIRST], [w_state#12, i_item_id#14, sales_before#28, sales_after#29] +(31) CometTakeOrderedAndProject +Input [4]: [w_state#12, i_item_id#14, sales_before#22, sales_after#23] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_state#12 ASC NULLS FIRST,i_item_id#14 ASC NULLS FIRST], output=[w_state#12,i_item_id#14,sales_before#22,sales_after#23]), [w_state#12, i_item_id#14, sales_before#22, sales_after#23], 100, [w_state#12 ASC NULLS FIRST, i_item_id#14 ASC NULLS FIRST], [w_state#12, i_item_id#14, sales_before#22, sales_after#23] + +(32) ColumnarToRow [codegen id : 1] +Input [4]: [w_state#12, i_item_id#14, sales_before#22, sales_after#23] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (37) -+- * ColumnarToRow (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.date_dim (34) +BroadcastExchange (36) ++- * ColumnarToRow (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) -(34) Scan parquet spark_catalog.default.date_dim +(33) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#16, d_date#17] 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 -(35) CometFilter +(34) CometFilter Input [2]: [d_date_sk#16, d_date#17] Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 2000-02-10)) AND (d_date#17 <= 2000-04-10)) AND isnotnull(d_date_sk#16)) -(36) ColumnarToRow [codegen id : 1] +(35) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#16, d_date#17] -(37) BroadcastExchange +(36) BroadcastExchange Input [2]: [d_date_sk#16, d_date#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/simplified.txt index 11c0201a8..6e47fa59f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/simplified.txt @@ -1,60 +1,41 @@ -TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] - WholeStageCodegen (9) - HashAggregate [w_state,i_item_id,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_date < 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END),sales_before,sales_after,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [w_state,i_item_id] #1 - WholeStageCodegen (8) - HashAggregate [w_state,i_item_id,d_date,cs_sales_price,cr_refunded_cash] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Project [cs_sales_price,cr_refunded_cash,w_state,i_item_id,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state] - BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - Project [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash] - SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - InputAdapter - WholeStageCodegen (2) - Sort [cs_order_number,cs_item_sk] - InputAdapter - Exchange [cs_order_number,cs_item_sk] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - WholeStageCodegen (4) - Sort [cr_order_number,cr_item_sk] - InputAdapter - Exchange [cr_order_number,cr_item_sk] #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk,w_state] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id,i_current_price] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] + CometHashAggregate [w_state,i_item_id,sales_before,sales_after,sum,isEmpty,sum,isEmpty,sum(CASE WHEN (d_date < 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END)] + CometColumnarExchange [w_state,i_item_id] #1 + CometHashAggregate [w_state,i_item_id,sum,isEmpty,sum,isEmpty,d_date,cs_sales_price,cr_refunded_cash] + CometProject [cs_sales_price,cr_refunded_cash,w_state,i_item_id,d_date] + CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_id,d_date_sk,d_date] + CometProject [cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_id] + CometBroadcastHashJoin [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_sk,i_item_id] + CometProject [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state] + CometBroadcastHashJoin [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_warehouse_sk,w_state] + CometProject [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash] + CometSortMergeJoin [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_refunded_cash] + CometSort [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] + CometColumnarExchange [cs_order_number,cs_item_sk] #2 + CometFilter [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [cr_item_sk,cr_order_number,cr_refunded_cash] + CometColumnarExchange [cr_order_number,cr_item_sk] #4 + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] + CometBroadcastExchange [w_warehouse_sk,w_state] #5 + CometFilter [w_warehouse_sk,w_state] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometProject [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id,i_current_price] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] + CometBroadcastExchange [d_date_sk,d_date] #7 + CometFilter [d_date_sk,d_date] + 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-spark3_5/q41/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/explain.txt index 07196ba8c..5c2b28b9d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/explain.txt @@ -1,24 +1,23 @@ == Physical Plan == -TakeOrderedAndProject (20) -+- * HashAggregate (19) - +- Exchange (18) - +- * HashAggregate (17) - +- * Project (16) - +- * BroadcastHashJoin Inner BuildRight (15) - :- * ColumnarToRow (4) - : +- CometProject (3) - : +- CometFilter (2) - : +- CometScan parquet spark_catalog.default.item (1) - +- BroadcastExchange (14) - +- * Project (13) - +- * Filter (12) - +- * HashAggregate (11) - +- Exchange (10) - +- * ColumnarToRow (9) - +- CometHashAggregate (8) - +- CometProject (7) - +- CometFilter (6) - +- CometScan parquet spark_catalog.default.item (5) +* ColumnarToRow (19) ++- CometTakeOrderedAndProject (18) + +- CometHashAggregate (17) + +- CometColumnarExchange (16) + +- CometHashAggregate (15) + +- CometProject (14) + +- CometBroadcastHashJoin (13) + :- CometProject (3) + : +- CometFilter (2) + : +- CometScan parquet spark_catalog.default.item (1) + +- CometBroadcastExchange (12) + +- CometProject (11) + +- CometFilter (10) + +- CometHashAggregate (9) + +- CometColumnarExchange (8) + +- CometHashAggregate (7) + +- CometProject (6) + +- CometFilter (5) + +- CometScan parquet spark_catalog.default.item (4) (1) Scan parquet spark_catalog.default.item @@ -36,84 +35,74 @@ Condition : (((isnotnull(i_manufact_id#1) AND (i_manufact_id#1 >= 738)) AND (i_m Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] Arguments: [i_manufact#2, i_product_name#3], [i_manufact#2, i_product_name#3] -(4) ColumnarToRow [codegen id : 3] -Input [2]: [i_manufact#2, i_product_name#3] - -(5) Scan parquet spark_catalog.default.item +(4) Scan parquet spark_catalog.default.item Output [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [Or(Or(And(EqualTo(i_category,Women ),Or(And(And(Or(EqualTo(i_color,powder ),EqualTo(i_color,khaki )),Or(EqualTo(i_units,Ounce ),EqualTo(i_units,Oz ))),Or(EqualTo(i_size,medium ),EqualTo(i_size,extra large ))),And(And(Or(EqualTo(i_color,brown ),EqualTo(i_color,honeydew )),Or(EqualTo(i_units,Bunch ),EqualTo(i_units,Ton ))),Or(EqualTo(i_size,N/A ),EqualTo(i_size,small ))))),And(EqualTo(i_category,Men ),Or(And(And(Or(EqualTo(i_color,floral ),EqualTo(i_color,deep )),Or(EqualTo(i_units,N/A ),EqualTo(i_units,Dozen ))),Or(EqualTo(i_size,petite ),EqualTo(i_size,large ))),And(And(Or(EqualTo(i_color,light ),EqualTo(i_color,cornflower )),Or(EqualTo(i_units,Box ),EqualTo(i_units,Pound ))),Or(EqualTo(i_size,medium ),EqualTo(i_size,extra large )))))),Or(And(EqualTo(i_category,Women ),Or(And(And(Or(EqualTo(i_color,midnight ),EqualTo(i_color,snow )),Or(EqualTo(i_units,Pallet ),EqualTo(i_units,Gross ))),Or(EqualTo(i_size,medium ),EqualTo(i_size,extra large ))),And(And(Or(EqualTo(i_color,cyan ),EqualTo(i_color,papaya )),Or(EqualTo(i_units,Cup ),EqualTo(i_units,Dram ))),Or(EqualTo(i_size,N/A ),EqualTo(i_size,small ))))),And(EqualTo(i_category,Men ),Or(And(And(Or(EqualTo(i_color,orange ),EqualTo(i_color,frosted )),Or(EqualTo(i_units,Each ),EqualTo(i_units,Tbl ))),Or(EqualTo(i_size,petite ),EqualTo(i_size,large ))),And(And(Or(EqualTo(i_color,forest ),EqualTo(i_color,ghost )),Or(EqualTo(i_units,Lb ),EqualTo(i_units,Bundle ))),Or(EqualTo(i_size,medium ),EqualTo(i_size,extra large ))))))), IsNotNull(i_manufact)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] Condition : (((((i_category#4 = Women ) AND (((((i_color#7 = powder ) OR (i_color#7 = khaki )) AND ((i_units#8 = Ounce ) OR (i_units#8 = Oz ))) AND ((i_size#6 = medium ) OR (i_size#6 = extra large ))) OR ((((i_color#7 = brown ) OR (i_color#7 = honeydew )) AND ((i_units#8 = Bunch ) OR (i_units#8 = Ton ))) AND ((i_size#6 = N/A ) OR (i_size#6 = small ))))) OR ((i_category#4 = Men ) AND (((((i_color#7 = floral ) OR (i_color#7 = deep )) AND ((i_units#8 = N/A ) OR (i_units#8 = Dozen ))) AND ((i_size#6 = petite ) OR (i_size#6 = large ))) OR ((((i_color#7 = light ) OR (i_color#7 = cornflower )) AND ((i_units#8 = Box ) OR (i_units#8 = Pound ))) AND ((i_size#6 = medium ) OR (i_size#6 = extra large )))))) OR (((i_category#4 = Women ) AND (((((i_color#7 = midnight ) OR (i_color#7 = snow )) AND ((i_units#8 = Pallet ) OR (i_units#8 = Gross ))) AND ((i_size#6 = medium ) OR (i_size#6 = extra large ))) OR ((((i_color#7 = cyan ) OR (i_color#7 = papaya )) AND ((i_units#8 = Cup ) OR (i_units#8 = Dram ))) AND ((i_size#6 = N/A ) OR (i_size#6 = small ))))) OR ((i_category#4 = Men ) AND (((((i_color#7 = orange ) OR (i_color#7 = frosted )) AND ((i_units#8 = Each ) OR (i_units#8 = Tbl ))) AND ((i_size#6 = petite ) OR (i_size#6 = large ))) OR ((((i_color#7 = forest ) OR (i_color#7 = ghost )) AND ((i_units#8 = Lb ) OR (i_units#8 = Bundle ))) AND ((i_size#6 = medium ) OR (i_size#6 = extra large ))))))) AND isnotnull(i_manufact#5)) -(7) CometProject +(6) CometProject Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] Arguments: [i_manufact#5], [i_manufact#5] -(8) CometHashAggregate +(7) CometHashAggregate Input [1]: [i_manufact#5] Keys [1]: [i_manufact#5] Functions [1]: [partial_count(1)] -(9) ColumnarToRow [codegen id : 1] -Input [2]: [i_manufact#5, count#9] - -(10) Exchange +(8) CometColumnarExchange Input [2]: [i_manufact#5, count#9] -Arguments: hashpartitioning(i_manufact#5, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: hashpartitioning(i_manufact#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(11) HashAggregate [codegen id : 2] +(9) CometHashAggregate Input [2]: [i_manufact#5, count#9] Keys [1]: [i_manufact#5] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#10] -Results [2]: [count(1)#10 AS item_cnt#11, i_manufact#5] -(12) Filter [codegen id : 2] -Input [2]: [item_cnt#11, i_manufact#5] -Condition : (item_cnt#11 > 0) +(10) CometFilter +Input [2]: [item_cnt#10, i_manufact#5] +Condition : (item_cnt#10 > 0) -(13) Project [codegen id : 2] -Output [1]: [i_manufact#5] -Input [2]: [item_cnt#11, i_manufact#5] +(11) CometProject +Input [2]: [item_cnt#10, i_manufact#5] +Arguments: [i_manufact#5], [i_manufact#5] -(14) BroadcastExchange +(12) CometBroadcastExchange Input [1]: [i_manufact#5] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] +Arguments: [i_manufact#5] -(15) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [i_manufact#2] -Right keys [1]: [i_manufact#5] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [2]: [i_manufact#2, i_product_name#3] +Right output [1]: [i_manufact#5] +Arguments: [i_manufact#2], [i_manufact#5], Inner, BuildRight -(16) Project [codegen id : 3] -Output [1]: [i_product_name#3] +(14) CometProject Input [3]: [i_manufact#2, i_product_name#3, i_manufact#5] +Arguments: [i_product_name#3], [i_product_name#3] -(17) HashAggregate [codegen id : 3] +(15) CometHashAggregate Input [1]: [i_product_name#3] Keys [1]: [i_product_name#3] Functions: [] -Aggregate Attributes: [] -Results [1]: [i_product_name#3] -(18) Exchange +(16) CometColumnarExchange Input [1]: [i_product_name#3] -Arguments: hashpartitioning(i_product_name#3, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(i_product_name#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(19) HashAggregate [codegen id : 4] +(17) CometHashAggregate Input [1]: [i_product_name#3] Keys [1]: [i_product_name#3] Functions: [] -Aggregate Attributes: [] -Results [1]: [i_product_name#3] -(20) TakeOrderedAndProject +(18) CometTakeOrderedAndProject +Input [1]: [i_product_name#3] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_product_name#3 ASC NULLS FIRST], output=[i_product_name#3]), [i_product_name#3], 100, [i_product_name#3 ASC NULLS FIRST], [i_product_name#3] + +(19) ColumnarToRow [codegen id : 1] Input [1]: [i_product_name#3] -Arguments: 100, [i_product_name#3 ASC NULLS FIRST], [i_product_name#3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/simplified.txt index 457d441ce..732e8679b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/simplified.txt @@ -1,29 +1,21 @@ -TakeOrderedAndProject [i_product_name] - WholeStageCodegen (4) - HashAggregate [i_product_name] - InputAdapter - Exchange [i_product_name] #1 - WholeStageCodegen (3) - HashAggregate [i_product_name] - Project [i_product_name] - BroadcastHashJoin [i_manufact,i_manufact] - ColumnarToRow - InputAdapter - CometProject [i_manufact,i_product_name] - CometFilter [i_manufact_id,i_manufact,i_product_name] - CometScan parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [i_manufact] - Filter [item_cnt] - HashAggregate [i_manufact,count] [count(1),item_cnt,count] - InputAdapter - Exchange [i_manufact] #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_manufact,count] - CometProject [i_manufact] - CometFilter [i_category,i_manufact,i_size,i_color,i_units] - CometScan parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_product_name] + CometHashAggregate [i_product_name] + CometColumnarExchange [i_product_name] #1 + CometHashAggregate [i_product_name] + CometProject [i_product_name] + CometBroadcastHashJoin [i_manufact,i_product_name,i_manufact] + CometProject [i_manufact,i_product_name] + CometFilter [i_manufact_id,i_manufact,i_product_name] + CometScan parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] + CometBroadcastExchange [i_manufact] #2 + CometProject [i_manufact] + CometFilter [item_cnt,i_manufact] + CometHashAggregate [item_cnt,i_manufact,count,count(1)] + CometColumnarExchange [i_manufact] #3 + CometHashAggregate [i_manufact,count] + CometProject [i_manufact] + CometFilter [i_category,i_manufact,i_size,i_color,i_units] + CometScan parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/explain.txt index 565b30e9a..ae0d6f333 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (19) -+- * HashAggregate (18) - +- Exchange (17) - +- * ColumnarToRow (16) +* ColumnarToRow (19) ++- CometTakeOrderedAndProject (18) + +- CometHashAggregate (17) + +- CometColumnarExchange (16) +- CometHashAggregate (15) +- CometProject (14) +- CometBroadcastHashJoin (13) @@ -93,21 +93,19 @@ 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))] -(16) ColumnarToRow [codegen id : 1] +(16) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(18) HashAggregate [codegen id : 2] +(17) CometHashAggregate 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))#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] -(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] +(18) CometTakeOrderedAndProject +Input [4]: [d_year#2, i_category_id#8, i_category#9, sum(ss_ext_sales_price)#12] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[sum(ss_ext_sales_price)#12 DESC NULLS LAST,d_year#2 ASC NULLS FIRST,i_category_id#8 ASC NULLS FIRST,i_category#9 ASC NULLS FIRST], output=[d_year#2,i_category_id#8,i_category#9,sum(ss_ext_sales_price)#12]), [d_year#2, i_category_id#8, i_category#9, sum(ss_ext_sales_price)#12], 100, [sum(ss_ext_sales_price)#12 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)#12] + +(19) ColumnarToRow [codegen id : 1] +Input [4]: [d_year#2, i_category_id#8, i_category#9, sum(ss_ext_sales_price)#12] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/simplified.txt index e3b850ef5..bdc71f6da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/simplified.txt @@ -1,23 +1,21 @@ -TakeOrderedAndProject [sum(ss_ext_sales_price),d_year,i_category_id,i_category] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [d_year,i_category_id,i_category,sum,ss_ext_sales_price] - CometProject [d_year,ss_ext_sales_price,i_category_id,i_category] - CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_category_id,i_category] - CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_category_id,i_category] #3 - CometProject [i_item_sk,i_category_id,i_category] - CometFilter [i_item_sk,i_category_id,i_category,i_manager_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_category_id,i_category,i_manager_id] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [d_year,i_category_id,i_category,sum(ss_ext_sales_price)] + CometHashAggregate [d_year,i_category_id,i_category,sum(ss_ext_sales_price),sum,sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange [d_year,i_category_id,i_category] #1 + CometHashAggregate [d_year,i_category_id,i_category,sum,ss_ext_sales_price] + CometProject [d_year,ss_ext_sales_price,i_category_id,i_category] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_category_id,i_category] + CometProject [d_year,ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometProject [d_date_sk,d_year] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange [i_item_sk,i_category_id,i_category] #3 + CometProject [i_item_sk,i_category_id,i_category] + CometFilter [i_item_sk,i_category_id,i_category,i_manager_id] + 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-spark3_5/q43/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/explain.txt index a2cfe7898..41a564299 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (19) -+- * HashAggregate (18) - +- Exchange (17) - +- * ColumnarToRow (16) +* ColumnarToRow (19) ++- CometTakeOrderedAndProject (18) + +- CometHashAggregate (17) + +- CometColumnarExchange (16) +- CometHashAggregate (15) +- CometProject (14) +- CometBroadcastHashJoin (13) @@ -93,21 +93,19 @@ 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))] -(16) ColumnarToRow [codegen id : 1] +(16) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(18) HashAggregate [codegen id : 2] +(17) CometHashAggregate 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))#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] -(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] +(18) CometTakeOrderedAndProject +Input [9]: [s_store_name#9, s_store_id#8, sun_sales#18, mon_sales#19, tue_sales#20, wed_sales#21, thu_sales#22, fri_sales#23, sat_sales#24] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#9 ASC NULLS FIRST,s_store_id#8 ASC NULLS FIRST,sun_sales#18 ASC NULLS FIRST,mon_sales#19 ASC NULLS FIRST,tue_sales#20 ASC NULLS FIRST,wed_sales#21 ASC NULLS FIRST,thu_sales#22 ASC NULLS FIRST,fri_sales#23 ASC NULLS FIRST,sat_sales#24 ASC NULLS FIRST], output=[s_store_name#9,s_store_id#8,sun_sales#18,mon_sales#19,tue_sales#20,wed_sales#21,thu_sales#22,fri_sales#23,sat_sales#24]), [s_store_name#9, s_store_id#8, sun_sales#18, mon_sales#19, tue_sales#20, wed_sales#21, thu_sales#22, fri_sales#23, sat_sales#24], 100, [s_store_name#9 ASC NULLS FIRST, s_store_id#8 ASC NULLS FIRST, sun_sales#18 ASC NULLS FIRST, mon_sales#19 ASC NULLS FIRST, tue_sales#20 ASC NULLS FIRST, wed_sales#21 ASC NULLS FIRST, thu_sales#22 ASC NULLS FIRST, fri_sales#23 ASC NULLS FIRST, sat_sales#24 ASC NULLS FIRST], [s_store_name#9, s_store_id#8, sun_sales#18, mon_sales#19, tue_sales#20, wed_sales#21, thu_sales#22, fri_sales#23, sat_sales#24] + +(19) ColumnarToRow [codegen id : 1] +Input [9]: [s_store_name#9, s_store_id#8, sun_sales#18, mon_sales#19, tue_sales#20, wed_sales#21, thu_sales#22, fri_sales#23, sat_sales#24] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/simplified.txt index 5a060b85c..6e1b8d6bf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/simplified.txt @@ -1,23 +1,21 @@ -TakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum,d_day_name,ss_sales_price] - CometProject [d_day_name,ss_sales_price,s_store_id,s_store_name] - CometBroadcastHashJoin [d_day_name,ss_store_sk,ss_sales_price,s_store_sk,s_store_id,s_store_name] - CometProject [d_day_name,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [d_date_sk,d_day_name,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_day_name] - CometFilter [d_date_sk,d_year,d_day_name] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] - CometBroadcastExchange [ss_store_sk,ss_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 - CometProject [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name,s_gmt_offset] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] + CometHashAggregate [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,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))] + CometColumnarExchange [s_store_name,s_store_id] #1 + CometHashAggregate [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum,d_day_name,ss_sales_price] + CometProject [d_day_name,ss_sales_price,s_store_id,s_store_name] + CometBroadcastHashJoin [d_day_name,ss_store_sk,ss_sales_price,s_store_sk,s_store_id,s_store_name] + CometProject [d_day_name,ss_store_sk,ss_sales_price] + CometBroadcastHashJoin [d_date_sk,d_day_name,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [d_date_sk,d_day_name] + CometFilter [d_date_sk,d_year,d_day_name] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] + CometBroadcastExchange [ss_store_sk,ss_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 + CometProject [s_store_sk,s_store_id,s_store_name] + CometFilter [s_store_sk,s_store_id,s_store_name,s_gmt_offset] + 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-spark3_5/q44/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/explain.txt index db7e47cce..9ee93656a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/explain.txt @@ -1,45 +1,50 @@ == Physical Plan == -TakeOrderedAndProject (41) -+- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (31) - : : +- * SortMergeJoin Inner (30) - : : :- * Sort (17) - : : : +- * Project (16) - : : : +- * Filter (15) - : : : +- Window (14) - : : : +- WindowGroupLimit (13) - : : : +- * Sort (12) - : : : +- Exchange (11) - : : : +- WindowGroupLimit (10) - : : : +- * Sort (9) - : : : +- * Filter (8) - : : : +- * HashAggregate (7) - : : : +- Exchange (6) - : : : +- * ColumnarToRow (5) - : : : +- CometHashAggregate (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : +- * Sort (29) - : : +- * Project (28) - : : +- * Filter (27) - : : +- Window (26) - : : +- WindowGroupLimit (25) - : : +- * Sort (24) - : : +- Exchange (23) - : : +- WindowGroupLimit (22) - : : +- * Sort (21) - : : +- * Filter (20) - : : +- * HashAggregate (19) - : : +- ReusedExchange (18) - : +- BroadcastExchange (35) - : +- * ColumnarToRow (34) - : +- CometFilter (33) - : +- CometScan parquet spark_catalog.default.item (32) - +- ReusedExchange (38) +TakeOrderedAndProject (46) ++- * Project (45) + +- * BroadcastHashJoin Inner BuildRight (44) + :- * Project (42) + : +- * BroadcastHashJoin Inner BuildRight (41) + : :- * Project (36) + : : +- * SortMergeJoin Inner (35) + : : :- * Sort (19) + : : : +- * Project (18) + : : : +- * Filter (17) + : : : +- Window (16) + : : : +- WindowGroupLimit (15) + : : : +- * ColumnarToRow (14) + : : : +- CometSort (13) + : : : +- CometColumnarExchange (12) + : : : +- RowToColumnar (11) + : : : +- WindowGroupLimit (10) + : : : +- * ColumnarToRow (9) + : : : +- CometSort (8) + : : : +- CometFilter (7) + : : : +- CometHashAggregate (6) + : : : +- CometColumnarExchange (5) + : : : +- CometHashAggregate (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- * Sort (34) + : : +- * Project (33) + : : +- * Filter (32) + : : +- Window (31) + : : +- WindowGroupLimit (30) + : : +- * ColumnarToRow (29) + : : +- CometSort (28) + : : +- CometColumnarExchange (27) + : : +- RowToColumnar (26) + : : +- WindowGroupLimit (25) + : : +- * ColumnarToRow (24) + : : +- CometSort (23) + : : +- CometFilter (22) + : : +- CometHashAggregate (21) + : : +- ReusedExchange (20) + : +- BroadcastExchange (40) + : +- * ColumnarToRow (39) + : +- CometFilter (38) + : +- CometScan parquet spark_catalog.default.item (37) + +- ReusedExchange (43) (1) Scan parquet spark_catalog.default.store_sales @@ -62,211 +67,220 @@ Input [2]: [ss_item_sk#1, ss_net_profit#3] Keys [1]: [ss_item_sk#1] Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#3))] -(5) ColumnarToRow [codegen id : 1] +(5) CometColumnarExchange Input [3]: [ss_item_sk#1, sum#5, count#6] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(6) Exchange -Input [3]: [ss_item_sk#1, sum#5, count#6] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] - -(7) HashAggregate [codegen id : 2] +(6) CometHashAggregate Input [3]: [ss_item_sk#1, sum#5, count#6] Keys [1]: [ss_item_sk#1] Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#3))#7] -Results [2]: [ss_item_sk#1 AS item_sk#8, cast((avg(UnscaledValue(ss_net_profit#3))#7 / 100.0) as decimal(11,6)) AS rank_col#9] -(8) Filter [codegen id : 2] -Input [2]: [item_sk#8, rank_col#9] -Condition : (isnotnull(rank_col#9) AND (cast(rank_col#9 as decimal(13,7)) > (0.9 * Subquery scalar-subquery#10, [id=#11]))) +(7) CometFilter +Input [2]: [item_sk#7, rank_col#8] +Condition : (isnotnull(rank_col#8) AND (cast(rank_col#8 as decimal(13,7)) > (0.9 * Subquery scalar-subquery#9, [id=#10]))) + +(8) CometSort +Input [2]: [item_sk#7, rank_col#8] +Arguments: [item_sk#7, rank_col#8], [rank_col#8 ASC NULLS FIRST] -(9) Sort [codegen id : 2] -Input [2]: [item_sk#8, rank_col#9] -Arguments: [rank_col#9 ASC NULLS FIRST], false, 0 +(9) ColumnarToRow [codegen id : 1] +Input [2]: [item_sk#7, rank_col#8] (10) WindowGroupLimit -Input [2]: [item_sk#8, rank_col#9] -Arguments: [rank_col#9 ASC NULLS FIRST], rank(rank_col#9), 10, Partial +Input [2]: [item_sk#7, rank_col#8] +Arguments: [rank_col#8 ASC NULLS FIRST], rank(rank_col#8), 10, Partial -(11) Exchange -Input [2]: [item_sk#8, rank_col#9] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] +(11) RowToColumnar +Input [2]: [item_sk#7, rank_col#8] -(12) Sort [codegen id : 3] -Input [2]: [item_sk#8, rank_col#9] -Arguments: [rank_col#9 ASC NULLS FIRST], false, 0 +(12) CometColumnarExchange +Input [2]: [item_sk#7, rank_col#8] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(13) WindowGroupLimit -Input [2]: [item_sk#8, rank_col#9] -Arguments: [rank_col#9 ASC NULLS FIRST], rank(rank_col#9), 10, Final +(13) CometSort +Input [2]: [item_sk#7, rank_col#8] +Arguments: [item_sk#7, rank_col#8], [rank_col#8 ASC NULLS FIRST] -(14) Window -Input [2]: [item_sk#8, rank_col#9] -Arguments: [rank(rank_col#9) windowspecdefinition(rank_col#9 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#12], [rank_col#9 ASC NULLS FIRST] +(14) ColumnarToRow [codegen id : 2] +Input [2]: [item_sk#7, rank_col#8] -(15) Filter [codegen id : 4] -Input [3]: [item_sk#8, rank_col#9, rnk#12] -Condition : ((rnk#12 < 11) AND isnotnull(item_sk#8)) +(15) WindowGroupLimit +Input [2]: [item_sk#7, rank_col#8] +Arguments: [rank_col#8 ASC NULLS FIRST], rank(rank_col#8), 10, Final -(16) Project [codegen id : 4] -Output [2]: [item_sk#8, rnk#12] -Input [3]: [item_sk#8, rank_col#9, rnk#12] +(16) Window +Input [2]: [item_sk#7, rank_col#8] +Arguments: [rank(rank_col#8) windowspecdefinition(rank_col#8 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#11], [rank_col#8 ASC NULLS FIRST] -(17) Sort [codegen id : 4] -Input [2]: [item_sk#8, rnk#12] -Arguments: [rnk#12 ASC NULLS FIRST], false, 0 +(17) Filter [codegen id : 3] +Input [3]: [item_sk#7, rank_col#8, rnk#11] +Condition : ((rnk#11 < 11) AND isnotnull(item_sk#7)) -(18) ReusedExchange [Reuses operator id: 6] -Output [3]: [ss_item_sk#13, sum#14, count#15] +(18) Project [codegen id : 3] +Output [2]: [item_sk#7, rnk#11] +Input [3]: [item_sk#7, rank_col#8, rnk#11] -(19) HashAggregate [codegen id : 6] -Input [3]: [ss_item_sk#13, sum#14, count#15] -Keys [1]: [ss_item_sk#13] -Functions [1]: [avg(UnscaledValue(ss_net_profit#16))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#16))#17] -Results [2]: [ss_item_sk#13 AS item_sk#18, cast((avg(UnscaledValue(ss_net_profit#16))#17 / 100.0) as decimal(11,6)) AS rank_col#19] +(19) Sort [codegen id : 3] +Input [2]: [item_sk#7, rnk#11] +Arguments: [rnk#11 ASC NULLS FIRST], false, 0 -(20) Filter [codegen id : 6] -Input [2]: [item_sk#18, rank_col#19] -Condition : (isnotnull(rank_col#19) AND (cast(rank_col#19 as decimal(13,7)) > (0.9 * ReusedSubquery Subquery scalar-subquery#10, [id=#11]))) +(20) ReusedExchange [Reuses operator id: 5] +Output [3]: [ss_item_sk#12, sum#13, count#14] -(21) Sort [codegen id : 6] -Input [2]: [item_sk#18, rank_col#19] -Arguments: [rank_col#19 DESC NULLS LAST], false, 0 +(21) CometHashAggregate +Input [3]: [ss_item_sk#12, sum#13, count#14] +Keys [1]: [ss_item_sk#12] +Functions [1]: [avg(UnscaledValue(ss_net_profit#15))] -(22) WindowGroupLimit -Input [2]: [item_sk#18, rank_col#19] -Arguments: [rank_col#19 DESC NULLS LAST], rank(rank_col#19), 10, Partial +(22) CometFilter +Input [2]: [item_sk#16, rank_col#17] +Condition : (isnotnull(rank_col#17) AND (cast(rank_col#17 as decimal(13,7)) > (0.9 * ReusedSubquery Subquery scalar-subquery#9, [id=#10]))) -(23) Exchange -Input [2]: [item_sk#18, rank_col#19] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] +(23) CometSort +Input [2]: [item_sk#16, rank_col#17] +Arguments: [item_sk#16, rank_col#17], [rank_col#17 DESC NULLS LAST] -(24) Sort [codegen id : 7] -Input [2]: [item_sk#18, rank_col#19] -Arguments: [rank_col#19 DESC NULLS LAST], false, 0 +(24) ColumnarToRow [codegen id : 4] +Input [2]: [item_sk#16, rank_col#17] (25) WindowGroupLimit -Input [2]: [item_sk#18, rank_col#19] -Arguments: [rank_col#19 DESC NULLS LAST], rank(rank_col#19), 10, Final +Input [2]: [item_sk#16, rank_col#17] +Arguments: [rank_col#17 DESC NULLS LAST], rank(rank_col#17), 10, Partial + +(26) RowToColumnar +Input [2]: [item_sk#16, rank_col#17] + +(27) CometColumnarExchange +Input [2]: [item_sk#16, rank_col#17] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(28) CometSort +Input [2]: [item_sk#16, rank_col#17] +Arguments: [item_sk#16, rank_col#17], [rank_col#17 DESC NULLS LAST] -(26) Window -Input [2]: [item_sk#18, rank_col#19] -Arguments: [rank(rank_col#19) windowspecdefinition(rank_col#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#20], [rank_col#19 DESC NULLS LAST] +(29) ColumnarToRow [codegen id : 5] +Input [2]: [item_sk#16, rank_col#17] -(27) Filter [codegen id : 8] -Input [3]: [item_sk#18, rank_col#19, rnk#20] -Condition : ((rnk#20 < 11) AND isnotnull(item_sk#18)) +(30) WindowGroupLimit +Input [2]: [item_sk#16, rank_col#17] +Arguments: [rank_col#17 DESC NULLS LAST], rank(rank_col#17), 10, Final -(28) Project [codegen id : 8] -Output [2]: [item_sk#18, rnk#20] -Input [3]: [item_sk#18, rank_col#19, rnk#20] +(31) Window +Input [2]: [item_sk#16, rank_col#17] +Arguments: [rank(rank_col#17) windowspecdefinition(rank_col#17 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#18], [rank_col#17 DESC NULLS LAST] -(29) Sort [codegen id : 8] -Input [2]: [item_sk#18, rnk#20] -Arguments: [rnk#20 ASC NULLS FIRST], false, 0 +(32) Filter [codegen id : 6] +Input [3]: [item_sk#16, rank_col#17, rnk#18] +Condition : ((rnk#18 < 11) AND isnotnull(item_sk#16)) -(30) SortMergeJoin [codegen id : 11] -Left keys [1]: [rnk#12] -Right keys [1]: [rnk#20] +(33) Project [codegen id : 6] +Output [2]: [item_sk#16, rnk#18] +Input [3]: [item_sk#16, rank_col#17, rnk#18] + +(34) Sort [codegen id : 6] +Input [2]: [item_sk#16, rnk#18] +Arguments: [rnk#18 ASC NULLS FIRST], false, 0 + +(35) SortMergeJoin [codegen id : 9] +Left keys [1]: [rnk#11] +Right keys [1]: [rnk#18] Join type: Inner Join condition: None -(31) Project [codegen id : 11] -Output [3]: [item_sk#8, rnk#12, item_sk#18] -Input [4]: [item_sk#8, rnk#12, item_sk#18, rnk#20] +(36) Project [codegen id : 9] +Output [3]: [item_sk#7, rnk#11, item_sk#16] +Input [4]: [item_sk#7, rnk#11, item_sk#16, rnk#18] -(32) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#21, i_product_name#22] +(37) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#19, i_product_name#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(33) CometFilter -Input [2]: [i_item_sk#21, i_product_name#22] -Condition : isnotnull(i_item_sk#21) +(38) CometFilter +Input [2]: [i_item_sk#19, i_product_name#20] +Condition : isnotnull(i_item_sk#19) -(34) ColumnarToRow [codegen id : 9] -Input [2]: [i_item_sk#21, i_product_name#22] +(39) ColumnarToRow [codegen id : 7] +Input [2]: [i_item_sk#19, i_product_name#20] -(35) BroadcastExchange -Input [2]: [i_item_sk#21, i_product_name#22] +(40) BroadcastExchange +Input [2]: [i_item_sk#19, i_product_name#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(36) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [item_sk#8] -Right keys [1]: [i_item_sk#21] +(41) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [item_sk#7] +Right keys [1]: [i_item_sk#19] Join type: Inner Join condition: None -(37) Project [codegen id : 11] -Output [3]: [rnk#12, item_sk#18, i_product_name#22] -Input [5]: [item_sk#8, rnk#12, item_sk#18, i_item_sk#21, i_product_name#22] +(42) Project [codegen id : 9] +Output [3]: [rnk#11, item_sk#16, i_product_name#20] +Input [5]: [item_sk#7, rnk#11, item_sk#16, i_item_sk#19, i_product_name#20] -(38) ReusedExchange [Reuses operator id: 35] -Output [2]: [i_item_sk#23, i_product_name#24] +(43) ReusedExchange [Reuses operator id: 40] +Output [2]: [i_item_sk#21, i_product_name#22] -(39) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [item_sk#18] -Right keys [1]: [i_item_sk#23] +(44) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [item_sk#16] +Right keys [1]: [i_item_sk#21] Join type: Inner Join condition: None -(40) Project [codegen id : 11] -Output [3]: [rnk#12, i_product_name#22 AS best_performing#25, i_product_name#24 AS worst_performing#26] -Input [5]: [rnk#12, item_sk#18, i_product_name#22, i_item_sk#23, i_product_name#24] +(45) Project [codegen id : 9] +Output [3]: [rnk#11, i_product_name#20 AS best_performing#23, i_product_name#22 AS worst_performing#24] +Input [5]: [rnk#11, item_sk#16, i_product_name#20, i_item_sk#21, i_product_name#22] -(41) TakeOrderedAndProject -Input [3]: [rnk#12, best_performing#25, worst_performing#26] -Arguments: 100, [rnk#12 ASC NULLS FIRST], [rnk#12, best_performing#25, worst_performing#26] +(46) TakeOrderedAndProject +Input [3]: [rnk#11, best_performing#23, worst_performing#24] +Arguments: 100, [rnk#11 ASC NULLS FIRST], [rnk#11, best_performing#23, worst_performing#24] ===== Subqueries ===== -Subquery:1 Hosting operator id = 8 Hosting Expression = Subquery scalar-subquery#10, [id=#11] -* HashAggregate (48) -+- Exchange (47) - +- * ColumnarToRow (46) - +- CometHashAggregate (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan parquet spark_catalog.default.store_sales (42) +Subquery:1 Hosting operator id = 7 Hosting Expression = Subquery scalar-subquery#9, [id=#10] +* ColumnarToRow (53) ++- CometHashAggregate (52) + +- CometColumnarExchange (51) + +- CometHashAggregate (50) + +- CometProject (49) + +- CometFilter (48) + +- CometScan parquet spark_catalog.default.store_sales (47) -(42) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_addr_sk#27, ss_store_sk#28, ss_net_profit#29, ss_sold_date_sk#30] +(47) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_addr_sk#25, ss_store_sk#26, ss_net_profit#27, ss_sold_date_sk#28] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)] ReadSchema: struct -(43) CometFilter -Input [4]: [ss_addr_sk#27, ss_store_sk#28, ss_net_profit#29, ss_sold_date_sk#30] -Condition : ((isnotnull(ss_store_sk#28) AND (ss_store_sk#28 = 4)) AND isnull(ss_addr_sk#27)) +(48) CometFilter +Input [4]: [ss_addr_sk#25, ss_store_sk#26, ss_net_profit#27, ss_sold_date_sk#28] +Condition : ((isnotnull(ss_store_sk#26) AND (ss_store_sk#26 = 4)) AND isnull(ss_addr_sk#25)) -(44) CometProject -Input [4]: [ss_addr_sk#27, ss_store_sk#28, ss_net_profit#29, ss_sold_date_sk#30] -Arguments: [ss_store_sk#28, ss_net_profit#29], [ss_store_sk#28, ss_net_profit#29] +(49) CometProject +Input [4]: [ss_addr_sk#25, ss_store_sk#26, ss_net_profit#27, ss_sold_date_sk#28] +Arguments: [ss_store_sk#26, ss_net_profit#27], [ss_store_sk#26, ss_net_profit#27] -(45) CometHashAggregate -Input [2]: [ss_store_sk#28, ss_net_profit#29] -Keys [1]: [ss_store_sk#28] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#29))] +(50) CometHashAggregate +Input [2]: [ss_store_sk#26, ss_net_profit#27] +Keys [1]: [ss_store_sk#26] +Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#27))] -(46) ColumnarToRow [codegen id : 1] -Input [3]: [ss_store_sk#28, sum#31, count#32] +(51) CometColumnarExchange +Input [3]: [ss_store_sk#26, sum#29, count#30] +Arguments: hashpartitioning(ss_store_sk#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(47) Exchange -Input [3]: [ss_store_sk#28, sum#31, count#32] -Arguments: hashpartitioning(ss_store_sk#28, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(52) CometHashAggregate +Input [3]: [ss_store_sk#26, sum#29, count#30] +Keys [1]: [ss_store_sk#26] +Functions [1]: [avg(UnscaledValue(ss_net_profit#27))] -(48) HashAggregate [codegen id : 2] -Input [3]: [ss_store_sk#28, sum#31, count#32] -Keys [1]: [ss_store_sk#28] -Functions [1]: [avg(UnscaledValue(ss_net_profit#29))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#29))#33] -Results [1]: [cast((avg(UnscaledValue(ss_net_profit#29))#33 / 100.0) as decimal(11,6)) AS rank_col#34] +(53) ColumnarToRow [codegen id : 1] +Input [1]: [rank_col#31] -Subquery:2 Hosting operator id = 20 Hosting Expression = ReusedSubquery Subquery scalar-subquery#10, [id=#11] +Subquery:2 Hosting operator id = 22 Hosting Expression = ReusedSubquery Subquery scalar-subquery#9, [id=#10] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/simplified.txt index 232d65d72..886ee5e0d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] - WholeStageCodegen (11) + WholeStageCodegen (9) Project [rnk,i_product_name,i_product_name] BroadcastHashJoin [item_sk,i_item_sk] Project [rnk,item_sk,i_product_name] @@ -7,66 +7,67 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] Project [item_sk,rnk,item_sk] SortMergeJoin [rnk,rnk] InputAdapter - WholeStageCodegen (4) + WholeStageCodegen (3) Sort [rnk] Project [item_sk,rnk] Filter [rnk,item_sk] InputAdapter Window [rank_col] WindowGroupLimit [rank_col] - WholeStageCodegen (3) - Sort [rank_col] + WholeStageCodegen (2) + ColumnarToRow InputAdapter - Exchange #1 - WindowGroupLimit [rank_col] - WholeStageCodegen (2) - Sort [rank_col] - Filter [rank_col] - Subquery #1 - WholeStageCodegen (2) - HashAggregate [ss_store_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),rank_col,sum,count] - InputAdapter - Exchange [ss_store_sk] #3 + CometSort [item_sk,rank_col] + CometColumnarExchange #1 + RowToColumnar + WindowGroupLimit [rank_col] + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometSort [item_sk,rank_col] + CometFilter [item_sk,rank_col] + Subquery #1 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_store_sk,sum,count,ss_net_profit] - CometProject [ss_store_sk,ss_net_profit] - CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,sum,count] - InputAdapter - Exchange [ss_item_sk] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + CometHashAggregate [rank_col,ss_store_sk,sum,count,avg(UnscaledValue(ss_net_profit))] + CometColumnarExchange [ss_store_sk] #3 + CometHashAggregate [ss_store_sk,sum,count,ss_net_profit] + CometProject [ss_store_sk,ss_net_profit] + CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometHashAggregate [item_sk,rank_col,ss_item_sk,sum,count,avg(UnscaledValue(ss_net_profit))] + CometColumnarExchange [ss_item_sk] #2 CometHashAggregate [ss_item_sk,sum,count,ss_net_profit] CometProject [ss_item_sk,ss_net_profit] CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] InputAdapter - WholeStageCodegen (8) + WholeStageCodegen (6) Sort [rnk] Project [item_sk,rnk] Filter [rnk,item_sk] InputAdapter Window [rank_col] WindowGroupLimit [rank_col] - WholeStageCodegen (7) - Sort [rank_col] + WholeStageCodegen (5) + ColumnarToRow InputAdapter - Exchange #4 - WindowGroupLimit [rank_col] - WholeStageCodegen (6) - Sort [rank_col] - Filter [rank_col] - ReusedSubquery [rank_col] #1 - HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,sum,count] + CometSort [item_sk,rank_col] + CometColumnarExchange #4 + RowToColumnar + WindowGroupLimit [rank_col] + WholeStageCodegen (4) + ColumnarToRow InputAdapter - ReusedExchange [ss_item_sk,sum,count] #2 + CometSort [item_sk,rank_col] + CometFilter [item_sk,rank_col] + ReusedSubquery [rank_col] #1 + CometHashAggregate [item_sk,rank_col,ss_item_sk,sum,count,avg(UnscaledValue(ss_net_profit))] + ReusedExchange [ss_item_sk,sum,count] #2 InputAdapter BroadcastExchange #5 - WholeStageCodegen (9) + WholeStageCodegen (7) ColumnarToRow InputAdapter CometFilter [i_item_sk,i_product_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/explain.txt index f128499e3..044247540 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/explain.txt @@ -1,40 +1,42 @@ == Physical Plan == -TakeOrderedAndProject (36) -+- * HashAggregate (35) - +- Exchange (34) - +- * HashAggregate (33) - +- * Project (32) - +- * Filter (31) - +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (30) - :- * 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) - +- * ColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan parquet spark_catalog.default.item (25) +TakeOrderedAndProject (38) ++- * HashAggregate (37) + +- * ColumnarToRow (36) + +- CometColumnarExchange (35) + +- RowToColumnar (34) + +- * HashAggregate (33) + +- * Project (32) + +- * Filter (31) + +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (30) + :- * 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) + +- * ColumnarToRow (28) + +- CometProject (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.item (25) (1) Scan parquet spark_catalog.default.web_sales @@ -195,50 +197,56 @@ Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] Aggregate Attributes [1]: [sum#19] Results [3]: [ca_zip#11, ca_city#10, sum#20] -(34) Exchange +(34) RowToColumnar 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 : 3] +(35) CometColumnarExchange +Input [3]: [ca_zip#11, ca_city#10, sum#20] +Arguments: hashpartitioning(ca_zip#11, ca_city#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(36) ColumnarToRow [codegen id : 3] +Input [3]: [ca_zip#11, ca_city#10, sum#20] + +(37) 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))#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 +(38) TakeOrderedAndProject 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 ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (41) -+- * ColumnarToRow (40) - +- CometProject (39) - +- CometFilter (38) - +- CometScan parquet spark_catalog.default.date_dim (37) +BroadcastExchange (43) ++- * ColumnarToRow (42) + +- CometProject (41) + +- CometFilter (40) + +- CometScan parquet spark_catalog.default.date_dim (39) -(37) Scan parquet spark_catalog.default.date_dim +(39) 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 -(38) CometFilter +(40) 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)) -(39) CometProject +(41) CometProject 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] +(42) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#12] -(41) BroadcastExchange +(43) BroadcastExchange Input [1]: [d_date_sk#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-spark3_5/q45/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/simplified.txt index f60fdb18a..8550b91c6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/simplified.txt @@ -1,51 +1,53 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] 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 (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] - ColumnarToRow - InputAdapter - CometProject [ws_sales_price,ca_city,ca_zip,i_item_id] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ca_city,ca_zip,i_item_sk,i_item_id] - CometProject [ws_item_sk,ws_sales_price,ca_city,ca_zip] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip,d_date_sk] - CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_city,ca_zip] - CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk,c_customer_sk,c_current_addr_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_city,ca_zip] #4 - CometFilter [ca_address_sk,ca_city,ca_zip] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + ColumnarToRow + InputAdapter + CometColumnarExchange [ca_zip,ca_city] #1 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometProject [ws_sales_price,ca_city,ca_zip,i_item_id] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ca_city,ca_zip,i_item_sk,i_item_id] + CometProject [ws_item_sk,ws_sales_price,ca_city,ca_zip] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip,d_date_sk] + CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_city,ca_zip] + CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk,c_customer_sk,c_current_addr_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange [ca_address_sk,ca_city,ca_zip] #4 + CometFilter [ca_address_sk,ca_city,ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [i_item_id] + CometFilter [i_item_sk,i_item_id] + 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-spark3_5/q46/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/explain.txt index fb99351a5..b81931528 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/explain.txt @@ -1,46 +1,42 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * Project (41) - +- * BroadcastHashJoin Inner BuildRight (40) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * HashAggregate (29) - : : +- Exchange (28) - : : +- * 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) - +- BroadcastExchange (39) - +- * ColumnarToRow (38) - +- CometFilter (37) - +- CometScan parquet spark_catalog.default.customer_address (36) +* ColumnarToRow (38) ++- CometTakeOrderedAndProject (37) + +- CometProject (36) + +- CometBroadcastHashJoin (35) + :- CometProject (33) + : +- CometBroadcastHashJoin (32) + : :- CometHashAggregate (28) + : : +- CometColumnarExchange (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) + : +- CometBroadcastExchange (31) + : +- CometFilter (30) + : +- CometScan parquet spark_catalog.default.customer (29) + +- ReusedExchange (34) (1) Scan parquet spark_catalog.default.store_sales @@ -168,110 +164,88 @@ Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, 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))] -(27) ColumnarToRow [codegen id : 1] +(27) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(28) Exchange -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 : 4] +(28) CometHashAggregate 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#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] +(29) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] 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)) - -(32) ColumnarToRow [codegen id : 2] -Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] - -(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=2] - -(34) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#27] -Join type: Inner -Join condition: None +(30) CometFilter +Input [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] +Condition : (isnotnull(c_customer_sk#22) AND isnotnull(c_current_addr_sk#23)) -(35) Project [codegen id : 4] -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] +(31) CometBroadcastExchange +Input [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] +Arguments: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] -(36) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#31, ca_city#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] -ReadSchema: struct +(32) CometBroadcastHashJoin +Left output [5]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#26, amt#27, profit#28] +Right output [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] +Arguments: [ss_customer_sk#1], [c_customer_sk#22], Inner, BuildRight -(37) CometFilter -Input [2]: [ca_address_sk#31, ca_city#32] -Condition : (isnotnull(ca_address_sk#31) AND isnotnull(ca_city#32)) +(33) CometProject +Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#26, amt#27, profit#28, c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] +Arguments: [ss_ticket_number#5, bought_city#26, amt#27, profit#28, c_current_addr_sk#23, c_first_name#24, c_last_name#25], [ss_ticket_number#5, bought_city#26, amt#27, profit#28, c_current_addr_sk#23, c_first_name#24, c_last_name#25] -(38) ColumnarToRow [codegen id : 3] -Input [2]: [ca_address_sk#31, ca_city#32] +(34) ReusedExchange [Reuses operator id: 23] +Output [2]: [ca_address_sk#29, ca_city#30] -(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] +(35) CometBroadcastHashJoin +Left output [7]: [ss_ticket_number#5, bought_city#26, amt#27, profit#28, c_current_addr_sk#23, c_first_name#24, c_last_name#25] +Right output [2]: [ca_address_sk#29, ca_city#30] +Arguments: [c_current_addr_sk#23], [ca_address_sk#29], Inner, NOT (ca_city#30 = bought_city#26), BuildRight -(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) +(36) CometProject +Input [9]: [ss_ticket_number#5, bought_city#26, amt#27, profit#28, c_current_addr_sk#23, c_first_name#24, c_last_name#25, ca_address_sk#29, ca_city#30] +Arguments: [c_last_name#25, c_first_name#24, ca_city#30, bought_city#26, ss_ticket_number#5, amt#27, profit#28], [c_last_name#25, c_first_name#24, ca_city#30, bought_city#26, ss_ticket_number#5, amt#27, profit#28] -(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] +(37) CometTakeOrderedAndProject +Input [7]: [c_last_name#25, c_first_name#24, ca_city#30, bought_city#26, ss_ticket_number#5, amt#27, profit#28] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#25 ASC NULLS FIRST,c_first_name#24 ASC NULLS FIRST,ca_city#30 ASC NULLS FIRST,bought_city#26 ASC NULLS FIRST,ss_ticket_number#5 ASC NULLS FIRST], output=[c_last_name#25,c_first_name#24,ca_city#30,bought_city#26,ss_ticket_number#5,amt#27,profit#28]), [c_last_name#25, c_first_name#24, ca_city#30, bought_city#26, ss_ticket_number#5, amt#27, profit#28], 100, [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, ca_city#30 ASC NULLS FIRST, bought_city#26 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#25, c_first_name#24, ca_city#30, bought_city#26, ss_ticket_number#5, amt#27, profit#28] -(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] +(38) ColumnarToRow [codegen id : 1] +Input [7]: [c_last_name#25, c_first_name#24, ca_city#30, 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 (47) -+- * ColumnarToRow (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan parquet spark_catalog.default.date_dim (43) +BroadcastExchange (43) ++- * ColumnarToRow (42) + +- CometProject (41) + +- CometFilter (40) + +- CometScan parquet spark_catalog.default.date_dim (39) -(43) Scan parquet spark_catalog.default.date_dim +(39) 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 -(44) CometFilter +(40) 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)) -(45) CometProject +(41) CometProject Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(46) ColumnarToRow [codegen id : 1] +(42) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(47) BroadcastExchange +(43) 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-spark3_5/q46/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/simplified.txt index 57defd961..1d28a8ede 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/simplified.txt @@ -1,60 +1,48 @@ -TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] - 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] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,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_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_address_sk,ca_city] - CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,hd_demo_sk] - CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,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_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [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] - 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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_city] - CometScan parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange [hd_demo_sk] #5 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [ca_address_sk,ca_city] #6 - CometFilter [ca_address_sk,ca_city] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_city] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] + CometProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] + CometBroadcastHashJoin [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk,ca_city] + CometProject [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name] + CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,bought_city,amt,profit,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometHashAggregate [ss_ticket_number,ss_customer_sk,bought_city,amt,profit,ss_addr_sk,ca_city,sum,sum,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] + CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,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_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_address_sk,ca_city] + CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,hd_demo_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,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_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [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] + 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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometBroadcastExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometBroadcastExchange [s_store_sk] #4 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_city] + CometScan parquet spark_catalog.default.store [s_store_sk,s_city] + CometBroadcastExchange [hd_demo_sk] #5 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange [ca_address_sk,ca_city] #6 + CometFilter [ca_address_sk,ca_city] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] #7 + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + ReusedExchange [ca_address_sk,ca_city] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/explain.txt index ce5faa952..3fe3003ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/explain.txt @@ -2,18 +2,18 @@ TakeOrderedAndProject (45) +- * Project (44) +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) + :- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) : :- * Project (28) : : +- * Filter (27) : : +- Window (26) : : +- * Filter (25) : : +- Window (24) - : : +- * Sort (23) - : : +- Exchange (22) - : : +- * HashAggregate (21) - : : +- Exchange (20) - : : +- * ColumnarToRow (19) + : : +- * ColumnarToRow (23) + : : +- CometSort (22) + : : +- CometColumnarExchange (21) + : : +- CometHashAggregate (20) + : : +- CometColumnarExchange (19) : : +- CometHashAggregate (18) : : +- CometProject (17) : : +- CometBroadcastHashJoin (16) @@ -32,18 +32,18 @@ TakeOrderedAndProject (45) : : +- CometBroadcastExchange (15) : : +- CometFilter (14) : : +- CometScan parquet spark_catalog.default.store (13) - : +- BroadcastExchange (35) - : +- * Project (34) - : +- Window (33) - : +- * Sort (32) - : +- Exchange (31) - : +- * HashAggregate (30) - : +- ReusedExchange (29) + : +- BroadcastExchange (34) + : +- * Project (33) + : +- Window (32) + : +- * ColumnarToRow (31) + : +- CometSort (30) + : +- ReusedExchange (29) +- BroadcastExchange (42) +- * Project (41) +- Window (40) - +- * Sort (39) - +- ReusedExchange (38) + +- * ColumnarToRow (39) + +- CometSort (38) + +- ReusedExchange (37) (1) Scan parquet spark_catalog.default.item @@ -135,120 +135,113 @@ Input [7]: [i_brand#2, i_category#3, ss_sales_price#6, d_year#10, d_moy#11, s_st 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))] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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#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 : 2] +(20) CometHashAggregate 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))#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#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] +(21) CometColumnarExchange +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, _w0#17] +Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(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 +(22) CometSort +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, _w0#17] +Arguments: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, _w0#17], [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] + +(23) ColumnarToRow [codegen id : 1] +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, _w0#17] (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#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] +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 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, 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#18], [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 : 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] +(25) Filter [codegen id : 2] +Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 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 [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 : 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 : 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#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum#27] - -(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#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 : 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#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 : 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#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 : 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)] +Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, _w0#17, rn#18] +Arguments: [avg(_w0#17) 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#19], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10] + +(27) Filter [codegen id : 7] +Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 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 : 7] +Output [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18] +Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, _w0#17, rn#18, avg_monthly_sales#19] + +(29) ReusedExchange [Reuses operator id: 21] +Output [7]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#16] + +(30) CometSort +Input [7]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#16] +Arguments: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#16], [i_category#20 ASC NULLS FIRST, i_brand#21 ASC NULLS FIRST, s_store_name#22 ASC NULLS FIRST, s_company_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] + +(31) ColumnarToRow [codegen id : 3] +Input [7]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#16] + +(32) Window +Input [7]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#16] +Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#20, i_brand#21, s_store_name#22, s_company_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#26], [i_category#20, i_brand#21, s_store_name#22, s_company_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] + +(33) Project [codegen id : 4] +Output [6]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, sum_sales#16 AS sum_sales#27, rn#26] +Input [8]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#16, rn#26] + +(34) BroadcastExchange +Input [6]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, sum_sales#27, rn#26] +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=3] + +(35) BroadcastHashJoin [codegen id : 7] +Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#18] +Right keys [5]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, (rn#26 + 1)] Join type: Inner Join condition: None -(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] +(36) Project [codegen id : 7] +Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, sum_sales#27] +Input [15]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, i_category#20, i_brand#21, s_store_name#22, s_company_name#23, sum_sales#27, rn#26] + +(37) ReusedExchange [Reuses operator id: 21] +Output [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#16] -(38) ReusedExchange [Reuses operator id: 31] -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] +(38) CometSort +Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#16] +Arguments: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#16], [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_name#30 ASC NULLS FIRST, s_company_name#31 ASC NULLS FIRST, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] -(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 +(39) ColumnarToRow [codegen id : 5] +Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#16] (40) Window -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] +Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#16] +Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#34], [i_category#28, i_brand#29, s_store_name#30, s_company_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] -(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] +(41) Project [codegen id : 6] +Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#16 AS sum_sales#35, rn#34] +Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#16, rn#34] (42) BroadcastExchange -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] +Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#35, rn#34] +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] -(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)] +(43) BroadcastHashJoin [codegen id : 7] +Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#18] +Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#34 - 1)] Join type: Inner Join condition: None -(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] +(44) Project [codegen id : 7] +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#19, sum_sales#16, sum_sales#27 AS psum#36, sum_sales#35 AS nsum#37] +Input [16]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, sum_sales#27, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#35, rn#34] (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#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] +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#19, sum_sales#16, psum#36, nsum#37] +Arguments: 100, [(sum_sales#16 - avg_monthly_sales#19) 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#19, sum_sales#16, psum#36, nsum#37] ===== Subqueries ===== @@ -275,6 +268,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=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/simplified.txt index b6e5c469b..91c2cd65d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/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 (13) + WholeStageCodegen (7) 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,67 +8,61 @@ 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 (4) + WholeStageCodegen (2) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (3) - Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum,ss_sales_price] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometFilter [i_item_sk,i_brand,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #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] + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 + CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum,sum(UnscaledValue(ss_sales_price))] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum,ss_sales_price] + CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #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 #7 - WholeStageCodegen (8) + WholeStageCodegen (4) 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 (7) - Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + WholeStageCodegen (3) + ColumnarToRow InputAdapter - 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 + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #1 InputAdapter - BroadcastExchange #9 - WholeStageCodegen (12) + BroadcastExchange #8 + WholeStageCodegen (6) 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 (11) - Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + WholeStageCodegen (5) + ColumnarToRow InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/explain.txt index 6b4d61a5e..3393afedf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* HashAggregate (28) -+- Exchange (27) - +- * ColumnarToRow (26) +* ColumnarToRow (28) ++- CometHashAggregate (27) + +- CometColumnarExchange (26) +- CometHashAggregate (25) +- CometProject (24) +- CometBroadcastHashJoin (23) @@ -150,19 +150,17 @@ Input [1]: [ss_quantity#4] Keys: [] Functions [1]: [partial_sum(ss_quantity#4)] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarExchange Input [1]: [sum#18] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(27) Exchange -Input [1]: [sum#18] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] - -(28) HashAggregate [codegen id : 2] +(27) CometHashAggregate Input [1]: [sum#18] 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] + +(28) ColumnarToRow [codegen id : 1] +Input [1]: [sum(ss_quantity)#19] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/simplified.txt index 5f628dbae..8bb0cede6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/simplified.txt @@ -1,40 +1,38 @@ -WholeStageCodegen (2) - HashAggregate [sum] [sum(ss_quantity),sum(ss_quantity),sum] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [sum,ss_quantity] - CometProject [ss_quantity] - CometBroadcastHashJoin [ss_quantity,ss_sold_date_sk,d_date_sk] - CometProject [ss_quantity,ss_sold_date_sk] - CometBroadcastHashJoin [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk,ca_address_sk,ca_state] - CometProject [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,cd_demo_sk,cd_marital_status,cd_education_status] - CometProject [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,s_store_sk] - CometFilter [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] - 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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk] #3 - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #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 [ca_address_sk,ca_state] #5 - CometProject [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometHashAggregate [sum(ss_quantity),sum,sum(ss_quantity)] + CometColumnarExchange #1 + CometHashAggregate [sum,ss_quantity] + CometProject [ss_quantity] + CometBroadcastHashJoin [ss_quantity,ss_sold_date_sk,d_date_sk] + CometProject [ss_quantity,ss_sold_date_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk,ca_address_sk,ca_state] + CometProject [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk] + CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,cd_demo_sk,cd_marital_status,cd_education_status] + CometProject [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] + CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,s_store_sk] + CometFilter [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] + 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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [s_store_sk] #3 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #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 [ca_address_sk,ca_state] #5 + CometProject [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + 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-spark3_5/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/explain.txt index 7df26543a..4b0d5aed0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/explain.txt @@ -1,81 +1,83 @@ == Physical Plan == -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) +* ColumnarToRow (79) ++- CometTakeOrderedAndProject (78) + +- CometHashAggregate (77) + +- CometColumnarExchange (76) + +- RowToColumnar (75) + +- * HashAggregate (74) + +- Union (73) + :- * Project (26) + : +- * Filter (25) + : +- Window (24) + : +- * Sort (23) + : +- Window (22) + : +- * ColumnarToRow (21) + : +- CometSort (20) + : +- CometColumnarExchange (19) + : +- CometHashAggregate (18) + : +- CometColumnarExchange (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) + : +- * ColumnarToRow (44) + : +- CometSort (43) + : +- CometColumnarExchange (42) + : +- CometHashAggregate (41) + : +- CometColumnarExchange (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) + +- * ColumnarToRow (67) + +- CometSort (66) + +- CometColumnarExchange (65) + +- CometHashAggregate (64) + +- CometColumnarExchange (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 @@ -155,312 +157,310 @@ Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, w 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))] -(17) ColumnarToRow [codegen id : 1] +(17) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(19) HashAggregate [codegen id : 2] +(18) CometHashAggregate 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))#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] -(20) Exchange -Input [3]: [item#26, return_ratio#27, currency_ratio#28] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] +(19) CometColumnarExchange +Input [3]: [item#22, return_ratio#23, currency_ratio#24] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(20) CometSort +Input [3]: [item#22, return_ratio#23, currency_ratio#24] +Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] -(21) Sort [codegen id : 3] -Input [3]: [item#26, return_ratio#27, currency_ratio#28] -Arguments: [return_ratio#27 ASC NULLS FIRST], false, 0 +(21) ColumnarToRow [codegen id : 1] +Input [3]: [item#22, return_ratio#23, currency_ratio#24] (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] +Input [3]: [item#22, return_ratio#23, currency_ratio#24] +Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] -(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 +(23) Sort [codegen id : 2] +Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] +Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 (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] +Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] +Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] -(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)) +(25) Filter [codegen id : 3] +Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] +Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) -(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] +(26) Project [codegen id : 3] +Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] (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] +Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#37), dynamicpruningexpression(cs_sold_date_sk#37 IN dynamicpruning#38)] +PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_sold_date_sk#33 IN dynamicpruning#34)] 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 (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)) +Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] +Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) (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] +Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] +Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] (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] +Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] +Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] (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] +Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] 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 (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)) +Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] +Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) (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] +Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] +Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] (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 +Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] +Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] +Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], 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] +Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] +Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] (36) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#44] +Output [1]: [d_date_sk#40] (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 +Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] +Right output [1]: [d_date_sk#40] +Arguments: [cs_sold_date_sk#33], [d_date_sk#40], 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] +Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] +Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] (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))] +Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] +Keys [1]: [cs_item_sk#28] +Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 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] +(40) CometColumnarExchange +Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] +Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(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] +(41) CometHashAggregate +Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] +Keys [1]: [cs_item_sk#28] +Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(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] +(42) CometColumnarExchange +Input [3]: [item#47, return_ratio#48, currency_ratio#49] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(43) Exchange -Input [3]: [item#55, return_ratio#56, currency_ratio#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +(43) CometSort +Input [3]: [item#47, return_ratio#48, currency_ratio#49] +Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] -(44) Sort [codegen id : 8] -Input [3]: [item#55, return_ratio#56, currency_ratio#57] -Arguments: [return_ratio#56 ASC NULLS FIRST], false, 0 +(44) ColumnarToRow [codegen id : 4] +Input [3]: [item#47, return_ratio#48, currency_ratio#49] (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] +Input [3]: [item#47, return_ratio#48, currency_ratio#49] +Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] -(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 +(46) Sort [codegen id : 5] +Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] +Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 (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] +Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] +Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] -(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)) +(48) Filter [codegen id : 6] +Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] +Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) -(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] +(49) Project [codegen id : 6] +Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] +Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] (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] +Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#66), dynamicpruningexpression(ss_sold_date_sk#66 IN dynamicpruning#67)] +PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] 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 (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)) +Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] +Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) (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] +Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] +Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] (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] +Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] +Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] (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] +Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] 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 (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)) +Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] +Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) (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] +Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] +Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] (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 +Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] +Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] +Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], 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] +Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] +Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] (59) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#73] +Output [1]: [d_date_sk#65] (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 +Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] +Right output [1]: [d_date_sk#65] +Arguments: [ss_sold_date_sk#58], [d_date_sk#65], 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] +Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] +Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] (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))] +Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] +Keys [1]: [ss_item_sk#53] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 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] +(63) CometColumnarExchange +Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] +Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(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] +(64) CometHashAggregate +Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] +Keys [1]: [ss_item_sk#53] +Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(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] +(65) CometColumnarExchange +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(66) Exchange -Input [3]: [item#84, return_ratio#85, currency_ratio#86] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +(66) CometSort +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] -(67) Sort [codegen id : 13] -Input [3]: [item#84, return_ratio#85, currency_ratio#86] -Arguments: [return_ratio#85 ASC NULLS FIRST], false, 0 +(67) ColumnarToRow [codegen id : 7] +Input [3]: [item#72, return_ratio#73, currency_ratio#74] (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] +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] -(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 +(69) Sort [codegen id : 8] +Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] +Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 (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] +Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] +Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] -(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)) +(71) Filter [codegen id : 9] +Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] +Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) -(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] +(72) Project [codegen id : 9] +Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] +Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] (73) Union -(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] +(74) HashAggregate [codegen id : 10] +Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(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] +(75) RowToColumnar +Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(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] +(76) CometColumnarExchange +Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(77) CometHashAggregate +Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] -(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] +(78) CometTakeOrderedAndProject +Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] + +(79) ColumnarToRow [codegen id : 11] +Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (82) -+- * ColumnarToRow (81) - +- CometProject (80) - +- CometFilter (79) - +- CometScan parquet spark_catalog.default.date_dim (78) +BroadcastExchange (84) ++- * ColumnarToRow (83) + +- CometProject (82) + +- CometFilter (81) + +- CometScan parquet spark_catalog.default.date_dim (80) -(78) Scan parquet spark_catalog.default.date_dim +(80) 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 -(79) CometFilter +(81) 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)) -(80) CometProject +(82) CometProject Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(81) ColumnarToRow [codegen id : 1] +(83) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(82) BroadcastExchange +(84) 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 = 27 Hosting Expression = cs_sold_date_sk#37 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#66 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/simplified.txt index 0e6b65b06..0d02a9c39 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/simplified.txt @@ -1,121 +1,111 @@ -TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] - WholeStageCodegen (17) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Exchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (16) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (5) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (4) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (3) - Sort [return_ratio] - InputAdapter - Exchange #2 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,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_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - 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 (9) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (8) - Sort [return_ratio] - InputAdapter - 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] #8 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometHashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,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_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_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] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - 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 (14) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (13) - Sort [return_ratio] - InputAdapter - 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] #11 - WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometHashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,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_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_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] - ReusedSubquery [d_date_sk] #1 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - ReusedExchange [d_date_sk] #6 +WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] + CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] + CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] + InputAdapter + Union + WholeStageCodegen (3) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (2) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #2 + CometHashAggregate [item,return_ratio,currency_ratio,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))] + CometColumnarExchange [ws_item_sk] #3 + CometHashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,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_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + WholeStageCodegen (6) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (5) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #7 + CometHashAggregate [item,return_ratio,currency_ratio,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))] + CometColumnarExchange [cs_item_sk] #8 + CometHashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,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_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_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] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (9) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (8) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #10 + CometHashAggregate [item,return_ratio,currency_ratio,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))] + CometColumnarExchange [ss_item_sk] #11 + CometHashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,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_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_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] + ReusedSubquery [d_date_sk] #1 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/explain.txt index 07bd5f03a..ed1f8feda 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/explain.txt @@ -1,74 +1,76 @@ == Physical Plan == -TakeOrderedAndProject (70) -+- * HashAggregate (69) - +- Exchange (68) - +- * HashAggregate (67) - +- * Expand (66) - +- Union (65) - :- * 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) - +- * 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) +TakeOrderedAndProject (72) ++- * HashAggregate (71) + +- * ColumnarToRow (70) + +- CometColumnarExchange (69) + +- RowToColumnar (68) + +- * HashAggregate (67) + +- * Expand (66) + +- Union (65) + :- * HashAggregate (22) + : +- * ColumnarToRow (21) + : +- CometColumnarExchange (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) + : +- * ColumnarToRow (40) + : +- CometColumnarExchange (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) + +- * ColumnarToRow (63) + +- CometColumnarExchange (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) (1) Scan parquet spark_catalog.default.store_sales @@ -164,14 +166,14 @@ 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))] -(20) ColumnarToRow [codegen id : 1] +(20) CometColumnarExchange Input [5]: [s_store_id#25, sum#26, sum#27, sum#28, sum#29] +Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(21) Exchange +(21) ColumnarToRow [codegen id : 1] 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] -(22) HashAggregate [codegen id : 2] +(22) HashAggregate [codegen id : 1] 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))] @@ -255,14 +257,14 @@ Input [5]: [sales_price#46, profit#47, return_amt#48, net_loss#49, cp_catalog_pa 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] +(39) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=2] -(40) Exchange +(40) ColumnarToRow [codegen id : 2] 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] +(41) HashAggregate [codegen id : 2] 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))] @@ -365,14 +367,14 @@ Input [5]: [sales_price#83, profit#84, return_amt#85, net_loss#86, web_site_id#1 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) ColumnarToRow [codegen id : 5] +(62) CometColumnarExchange Input [5]: [web_site_id#104, sum#105, sum#106, sum#107, sum#108] +Arguments: hashpartitioning(web_site_id#104, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(63) Exchange +(63) ColumnarToRow [codegen id : 3] 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 : 6] +(64) HashAggregate [codegen id : 3] 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))] @@ -381,61 +383,67 @@ Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#83))#109,17,2) AS sales# (65) Union -(66) Expand [codegen id : 7] +(66) Expand [codegen id : 4] 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 : 7] +(67) HashAggregate [codegen id : 4] 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 +(68) RowToColumnar 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 : 8] +(69) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=4] + +(70) ColumnarToRow [codegen id : 5] +Input [9]: [channel#118, id#119, spark_grouping_id#120, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] + +(71) HashAggregate [codegen id : 5] 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 +(72) TakeOrderedAndProject 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 ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (75) -+- * ColumnarToRow (74) - +- CometProject (73) - +- CometFilter (72) - +- CometScan parquet spark_catalog.default.date_dim (71) +BroadcastExchange (77) ++- * ColumnarToRow (76) + +- CometProject (75) + +- CometFilter (74) + +- CometScan parquet spark_catalog.default.date_dim (73) -(71) Scan parquet spark_catalog.default.date_dim +(73) 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 -(72) CometFilter +(74) 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)) -(73) CometProject +(75) CometProject Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(74) ColumnarToRow [codegen id : 1] +(76) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(75) BroadcastExchange +(77) BroadcastExchange Input [1]: [d_date_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/simplified.txt index 537cba446..7b4b411a4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/simplified.txt @@ -1,20 +1,20 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (8) + WholeStageCodegen (5) 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 (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 (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 (1) - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometColumnarExchange [channel,id,spark_grouping_id] #1 + RowToColumnar + WholeStageCodegen (4) + 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 (1) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [s_store_id] #2 CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,s_store_id] CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] @@ -43,13 +43,11 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometBroadcastExchange [s_store_sk,s_store_id] #5 CometFilter [s_store_sk,s_store_id] 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] #6 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter + WholeStageCodegen (2) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [cp_catalog_page_id] #6 CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] @@ -68,13 +66,11 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #7 CometFilter [cp_catalog_page_sk,cp_catalog_page_id] 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] #8 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter + WholeStageCodegen (3) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [web_site_id] #8 CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,web_site_id] CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/explain.txt index c1c7074be..1e9d7743c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (28) -+- * HashAggregate (27) - +- Exchange (26) - +- * ColumnarToRow (25) +* ColumnarToRow (28) ++- CometTakeOrderedAndProject (27) + +- CometHashAggregate (26) + +- CometColumnarExchange (25) +- CometHashAggregate (24) +- CometProject (23) +- CometBroadcastHashJoin (22) @@ -147,23 +147,21 @@ Input [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_compan 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)] -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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#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 : 2] +(26) CometHashAggregate 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)#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 #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] +(27) CometTakeOrderedAndProject +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 #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[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], output=[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 #31,31 - 60 days #32,61 - 90 days #33,91 - 120 days #34,>120 days #35]), [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 #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35], 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 #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] + +(28) 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, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/simplified.txt index 58adabcec..075786f9b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/simplified.txt @@ -1,40 +1,38 @@ -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 (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 (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,sum,sum,sum,sum,sum,sr_returned_date_sk,ss_sold_date_sk] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [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 ] + 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,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,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)] + CometColumnarExchange [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 + 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,sum,sum,sum,sum,sum,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 [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,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,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,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,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,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,ss_sold_date_sk,sr_returned_date_sk,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] - CometProject [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] #2 - CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [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] #4 - CometFilter [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] - 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 [d_date_sk] #5 - CometFilter [d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastHashJoin [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk,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] + CometProject [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] #2 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_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_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [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] #4 + CometFilter [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] + 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 [d_date_sk] #5 + CometFilter [d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + 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-spark3_5/q51/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/explain.txt index bc106394c..3bd86214d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/explain.txt @@ -1,44 +1,47 @@ == Physical Plan == -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) +* ColumnarToRow (43) ++- CometTakeOrderedAndProject (42) + +- CometFilter (41) + +- CometWindowExec (40) + +- CometSort (39) + +- CometColumnarExchange (38) + +- CometProject (37) + +- CometSortMergeJoin (36) + :- CometSort (19) + : +- CometColumnarExchange (18) + : +- RowToColumnar (17) + : +- * Project (16) + : +- Window (15) + : +- * ColumnarToRow (14) + : +- CometSort (13) + : +- CometColumnarExchange (12) + : +- CometHashAggregate (11) + : +- CometColumnarExchange (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) + +- CometSort (35) + +- CometColumnarExchange (34) + +- RowToColumnar (33) + +- * Project (32) + +- Window (31) + +- * ColumnarToRow (30) + +- CometSort (29) + +- CometColumnarExchange (28) + +- CometHashAggregate (27) + +- CometColumnarExchange (26) + +- CometHashAggregate (25) + +- CometProject (24) + +- CometBroadcastHashJoin (23) + :- CometFilter (21) + : +- CometScan parquet spark_catalog.default.store_sales (20) + +- ReusedExchange (22) (1) Scan parquet spark_catalog.default.web_sales @@ -86,173 +89,177 @@ 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))] -(10) ColumnarToRow [codegen id : 1] +(10) CometColumnarExchange Input [3]: [ws_item_sk#1, d_date#6, sum#8] +Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(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] - -(12) HashAggregate [codegen id : 2] +(11) CometHashAggregate 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] -(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] +(12) CometColumnarExchange +Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(13) CometSort +Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] +Arguments: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(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 +(14) ColumnarToRow [codegen id : 1] +Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] (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] +Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] +Arguments: [sum(_w0#10) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(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] +(16) Project [codegen id : 2] +Output [3]: [item_sk#9, d_date#6, cume_sales#11] +Input [5]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1, cume_sales#11] -(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] +(17) RowToColumnar +Input [3]: [item_sk#9, d_date#6, cume_sales#11] -(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 +(18) CometColumnarExchange +Input [3]: [item_sk#9, d_date#6, cume_sales#11] +Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(19) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] +(19) CometSort +Input [3]: [item_sk#9, d_date#6, cume_sales#11] +Arguments: [item_sk#9, d_date#6, cume_sales#11], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] + +(20) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] 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#14), dynamicpruningexpression(ss_sold_date_sk#14 IN dynamicpruning#15)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(20) CometFilter -Input [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] -Condition : isnotnull(ss_item_sk#13) +(21) CometFilter +Input [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] +Condition : isnotnull(ss_item_sk#12) + +(22) ReusedExchange [Reuses operator id: 6] +Output [2]: [d_date_sk#16, d_date#17] + +(23) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] +Right output [2]: [d_date_sk#16, d_date#17] +Arguments: [ss_sold_date_sk#14], [d_date_sk#16], Inner, BuildRight -(21) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#17, d_date#18] +(24) CometProject +Input [5]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14, d_date_sk#16, d_date#17] +Arguments: [ss_item_sk#12, ss_sales_price#13, d_date#17], [ss_item_sk#12, ss_sales_price#13, d_date#17] -(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 +(25) CometHashAggregate +Input [3]: [ss_item_sk#12, ss_sales_price#13, d_date#17] +Keys [2]: [ss_item_sk#12, d_date#17] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#13))] -(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] +(26) CometColumnarExchange +Input [3]: [ss_item_sk#12, d_date#17, sum#18] +Arguments: hashpartitioning(ss_item_sk#12, d_date#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(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))] +(27) CometHashAggregate +Input [3]: [ss_item_sk#12, d_date#17, sum#18] +Keys [2]: [ss_item_sk#12, d_date#17] +Functions [1]: [sum(UnscaledValue(ss_sales_price#13))] -(25) ColumnarToRow [codegen id : 6] -Input [3]: [ss_item_sk#13, d_date#18, sum#19] +(28) CometColumnarExchange +Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] +Arguments: hashpartitioning(ss_item_sk#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(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] +(29) CometSort +Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] +Arguments: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12], [ss_item_sk#12 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] -(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))#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] +(30) ColumnarToRow [codegen id : 3] +Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] -(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] +(31) Window +Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] +Arguments: [sum(_w0#20) windowspecdefinition(ss_item_sk#12, d_date#17 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#21], [ss_item_sk#12], [d_date#17 ASC NULLS FIRST] -(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 +(32) Project [codegen id : 4] +Output [3]: [item_sk#19, d_date#17, cume_sales#21] +Input [5]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12, cume_sales#21] -(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] +(33) RowToColumnar +Input [3]: [item_sk#19, d_date#17, cume_sales#21] -(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] +(34) CometColumnarExchange +Input [3]: [item_sk#19, d_date#17, cume_sales#21] +Arguments: hashpartitioning(item_sk#19, d_date#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [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] +(35) CometSort +Input [3]: [item_sk#19, d_date#17, cume_sales#21] +Arguments: [item_sk#19, d_date#17, cume_sales#21], [item_sk#19 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] -(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 +(36) CometSortMergeJoin +Left output [3]: [item_sk#9, d_date#6, cume_sales#11] +Right output [3]: [item_sk#19, d_date#17, cume_sales#21] +Arguments: [item_sk#9, d_date#6], [item_sk#19, d_date#17], FullOuter -(34) SortMergeJoin [codegen id : 11] -Left keys [2]: [item_sk#10, d_date#6] -Right keys [2]: [item_sk#21, d_date#18] -Join type: FullOuter -Join condition: None +(37) CometProject +Input [6]: [item_sk#9, d_date#6, cume_sales#11, item_sk#19, d_date#17, cume_sales#21] +Arguments: [item_sk#22, d_date#23, web_sales#24, store_sales#25], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#19 END AS item_sk#22, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#17 END AS d_date#23, cume_sales#11 AS web_sales#24, cume_sales#21 AS store_sales#25] -(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] +(38) CometColumnarExchange +Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] +Arguments: hashpartitioning(item_sk#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [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] +(39) CometSort +Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] +Arguments: [item_sk#22, d_date#23, web_sales#24, store_sales#25], [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST] -(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 +(40) CometWindowExec +Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] +Arguments: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27], [max(web_sales#24) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#26, max(store_sales#25) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#27], [item_sk#22], [d_date#23 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] +(41) CometFilter +Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] +Condition : ((isnotnull(web_cumulative#26) AND isnotnull(store_cumulative#27)) AND (web_cumulative#26 > store_cumulative#27)) -(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)) +(42) CometTakeOrderedAndProject +Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_sk#22 ASC NULLS FIRST,d_date#23 ASC NULLS FIRST], output=[item_sk#22,d_date#23,web_sales#24,store_sales#25,web_cumulative#26,store_cumulative#27]), [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27], 100, [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST], [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] -(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] +(43) ColumnarToRow [codegen id : 5] +Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -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 +(44) 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 -(42) CometFilter +(45) 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)) -(43) CometProject +(46) 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] -(44) ColumnarToRow [codegen id : 1] +(47) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(45) BroadcastExchange +(48) 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 = 19 Hosting Expression = ss_sold_date_sk#15 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/simplified.txt index 988297f02..0e5a8e3d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/simplified.txt @@ -1,76 +1,62 @@ -TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (13) - Filter [web_cumulative,store_cumulative] - InputAdapter - Window [web_sales,item_sk,d_date,store_sales] - WholeStageCodegen (12) - Sort [item_sk,d_date] - InputAdapter - Exchange [item_sk] #1 - 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 (5) - Sort [item_sk,d_date] +WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + CometFilter [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + CometWindowExec [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + CometSort [item_sk,d_date,web_sales,store_sales] + CometColumnarExchange [item_sk] #1 + CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] + CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #2 + RowToColumnar + WholeStageCodegen (2) + Project [item_sk,d_date,cume_sales] InputAdapter - Exchange [item_sk,d_date] #2 - WholeStageCodegen (4) - Project [item_sk,d_date,cume_sales] + Window [_w0,ws_item_sk,d_date] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Window [_w0,ws_item_sk,d_date] - WholeStageCodegen (3) - Sort [ws_item_sk,d_date] - InputAdapter - Exchange [ws_item_sk] #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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ws_item_sk,d_date,sum,ws_sales_price] - CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #6 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - WholeStageCodegen (10) - Sort [item_sk,d_date] + CometSort [item_sk,d_date,_w0,ws_item_sk] + CometColumnarExchange [ws_item_sk] #3 + CometHashAggregate [item_sk,d_date,_w0,ws_item_sk,sum,sum(UnscaledValue(ws_sales_price))] + CometColumnarExchange [ws_item_sk,d_date] #4 + CometHashAggregate [ws_item_sk,d_date,sum,ws_sales_price] + CometProject [ws_item_sk,ws_sales_price,d_date] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange [d_date_sk,d_date] #6 + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #7 + RowToColumnar + WholeStageCodegen (4) + Project [item_sk,d_date,cume_sales] InputAdapter - Exchange [item_sk,d_date] #7 - WholeStageCodegen (9) - Project [item_sk,d_date,cume_sales] + Window [_w0,ss_item_sk,d_date] + WholeStageCodegen (3) + ColumnarToRow InputAdapter - Window [_w0,ss_item_sk,d_date] - WholeStageCodegen (8) - Sort [ss_item_sk,d_date] - InputAdapter - 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] #9 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometHashAggregate [ss_item_sk,d_date,sum,ss_sales_price] - CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_date] #6 + CometSort [item_sk,d_date,_w0,ss_item_sk] + CometColumnarExchange [ss_item_sk] #8 + CometHashAggregate [item_sk,d_date,_w0,ss_item_sk,sum,sum(UnscaledValue(ss_sales_price))] + CometColumnarExchange [ss_item_sk,d_date] #9 + CometHashAggregate [ss_item_sk,d_date,sum,ss_sales_price] + CometProject [ss_item_sk,ss_sales_price,d_date] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_date] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/explain.txt index 65c777125..f8beff744 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (19) -+- * HashAggregate (18) - +- Exchange (17) - +- * ColumnarToRow (16) +* ColumnarToRow (19) ++- CometTakeOrderedAndProject (18) + +- CometHashAggregate (17) + +- CometColumnarExchange (16) +- CometHashAggregate (15) +- CometProject (14) +- CometBroadcastHashJoin (13) @@ -93,21 +93,19 @@ 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))] -(16) ColumnarToRow [codegen id : 1] +(16) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(18) HashAggregate [codegen id : 2] +(17) CometHashAggregate 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))#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] -(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] +(18) CometTakeOrderedAndProject +Input [4]: [d_year#2, brand_id#12, brand#13, ext_price#14] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,ext_price#14 DESC NULLS LAST,brand_id#12 ASC NULLS FIRST], output=[d_year#2,brand_id#12,brand#13,ext_price#14]), [d_year#2, brand_id#12, brand#13, ext_price#14], 100, [d_year#2 ASC NULLS FIRST, ext_price#14 DESC NULLS LAST, brand_id#12 ASC NULLS FIRST], [d_year#2, brand_id#12, brand#13, ext_price#14] + +(19) ColumnarToRow [codegen id : 1] +Input [4]: [d_year#2, brand_id#12, brand#13, ext_price#14] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/simplified.txt index a0932c7cc..99f3baf91 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/simplified.txt @@ -1,23 +1,21 @@ -TakeOrderedAndProject [d_year,ext_price,brand_id,brand] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [d_year,i_brand,i_brand_id,sum,ss_ext_sales_price] - CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] - CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [d_year,brand_id,brand,ext_price] + CometHashAggregate [d_year,brand_id,brand,ext_price,i_brand,i_brand_id,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange [d_year,i_brand,i_brand_id] #1 + CometHashAggregate [d_year,i_brand,i_brand_id,sum,ss_ext_sales_price] + CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] + CometProject [d_year,ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometProject [d_date_sk,d_year] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 + CometProject [i_item_sk,i_brand_id,i_brand] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] + 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-spark3_5/q53/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/explain.txt index 76d80c043..71c84c040 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/explain.txt @@ -3,11 +3,11 @@ TakeOrderedAndProject (29) +- * Project (28) +- * Filter (27) +- Window (26) - +- * Sort (25) - +- Exchange (24) - +- * HashAggregate (23) - +- Exchange (22) - +- * ColumnarToRow (21) + +- * ColumnarToRow (25) + +- CometSort (24) + +- CometColumnarExchange (23) + +- CometHashAggregate (22) + +- CometColumnarExchange (21) +- CometHashAggregate (20) +- CometProject (19) +- CometBroadcastHashJoin (18) @@ -127,43 +127,41 @@ 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))] -(21) ColumnarToRow [codegen id : 1] +(21) CometColumnarExchange Input [3]: [i_manufact_id#5, d_qoy#17, sum#19] +Arguments: hashpartitioning(i_manufact_id#5, d_qoy#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(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] - -(23) HashAggregate [codegen id : 2] +(22) CometHashAggregate 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] -(24) Exchange -Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] -Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(23) CometColumnarExchange +Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] +Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(24) CometSort +Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] +Arguments: [i_manufact_id#5, sum_sales#20, _w0#21], [i_manufact_id#5 ASC NULLS FIRST] -(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) ColumnarToRow [codegen id : 1] +Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] (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] +Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] +Arguments: [avg(_w0#21) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#22], [i_manufact_id#5] -(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) Filter [codegen id : 2] +Input [4]: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22] +Condition : CASE WHEN (avg_quarterly_sales#22 > 0.000000) THEN ((abs((sum_sales#20 - avg_quarterly_sales#22)) / avg_quarterly_sales#22) > 0.1000000000000000) ELSE false END -(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) Project [codegen id : 2] +Output [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] +Input [4]: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22] (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] +Input [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] +Arguments: 100, [avg_quarterly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/simplified.txt index b2199fb34..a0d09de86 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/simplified.txt @@ -1,45 +1,41 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] - WholeStageCodegen (4) + WholeStageCodegen (2) Project [i_manufact_id,sum_sales,avg_quarterly_sales] Filter [avg_quarterly_sales,sum_sales] InputAdapter Window [_w0,i_manufact_id] - WholeStageCodegen (3) - Sort [i_manufact_id] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [i_manufact_id] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_manufact_id,d_qoy,sum,ss_sales_price] - CometProject [i_manufact_id,ss_sales_price,d_qoy] - CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] - CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] - CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] - CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_item_sk,i_manufact_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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_date_sk,d_month_seq,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [d_date_sk,d_qoy] #5 - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] + CometSort [i_manufact_id,sum_sales,_w0] + CometColumnarExchange [i_manufact_id] #1 + CometHashAggregate [i_manufact_id,sum_sales,_w0,d_qoy,sum,sum(UnscaledValue(ss_sales_price))] + CometColumnarExchange [i_manufact_id,d_qoy] #2 + CometHashAggregate [i_manufact_id,d_qoy,sum,ss_sales_price] + CometProject [i_manufact_id,ss_sales_price,d_qoy] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] + CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] + CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_item_sk,i_manufact_id] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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_date_sk,d_month_seq,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometBroadcastExchange [d_date_sk,d_qoy] #5 + CometProject [d_date_sk,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometBroadcastExchange [s_store_sk] #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-spark3_5/q54/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/explain.txt index 9eca5ceea..cf1531550 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/explain.txt @@ -1,60 +1,60 @@ == Physical Plan == -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) +* ColumnarToRow (56) ++- CometTakeOrderedAndProject (55) + +- CometHashAggregate (54) + +- CometColumnarExchange (53) + +- CometHashAggregate (52) + +- CometHashAggregate (51) + +- CometColumnarExchange (50) + +- CometHashAggregate (49) + +- CometProject (48) + +- CometBroadcastHashJoin (47) + :- CometProject (42) + : +- CometBroadcastHashJoin (41) + : :- CometProject (37) + : : +- CometBroadcastHashJoin (36) + : : :- CometProject (32) + : : : +- CometBroadcastHashJoin (31) + : : : :- CometHashAggregate (27) + : : : : +- CometColumnarExchange (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) + : : : +- CometBroadcastExchange (30) + : : : +- CometFilter (29) + : : : +- CometScan parquet spark_catalog.default.store_sales (28) + : : +- CometBroadcastExchange (35) + : : +- CometFilter (34) + : : +- CometScan parquet spark_catalog.default.customer_address (33) + : +- CometBroadcastExchange (40) + : +- CometFilter (39) + : +- CometScan parquet spark_catalog.default.store (38) + +- CometBroadcastExchange (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) (1) Scan parquet spark_catalog.default.catalog_sales @@ -178,21 +178,16 @@ Input [2]: [c_customer_sk#20, c_current_addr_sk#21] Keys [2]: [c_customer_sk#20, c_current_addr_sk#21] Functions: [] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(27) 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#20, c_current_addr_sk#21] -(29) Scan parquet spark_catalog.default.store_sales +(28) 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 [] @@ -200,135 +195,133 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#24), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(30) CometFilter +(29) CometFilter Input [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] Condition : isnotnull(ss_customer_sk#22) -(31) ColumnarToRow [codegen id : 2] -Input [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] - -(32) BroadcastExchange +(30) CometBroadcastExchange 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] +Arguments: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -(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 +(31) CometBroadcastHashJoin +Left output [2]: [c_customer_sk#20, c_current_addr_sk#21] +Right output [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] +Arguments: [c_customer_sk#20], [ss_customer_sk#22], Inner, BuildRight -(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] +(32) CometProject 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] +Arguments: [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24], [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24] -(35) Scan parquet spark_catalog.default.customer_address +(33) 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 -(36) CometFilter +(34) 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)) -(37) ColumnarToRow [codegen id : 3] +(35) CometBroadcastExchange Input [3]: [ca_address_sk#26, ca_county#27, ca_state#28] +Arguments: [ca_address_sk#26, ca_county#27, ca_state#28] -(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] - -(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 +(36) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24] +Right output [3]: [ca_address_sk#26, ca_county#27, ca_state#28] +Arguments: [c_current_addr_sk#21], [ca_address_sk#26], Inner, BuildRight -(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] +(37) CometProject 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] +Arguments: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#28], [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#28] -(41) Scan parquet spark_catalog.default.store +(38) 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 -(42) CometFilter +(39) CometFilter Input [2]: [s_county#29, s_state#30] Condition : (isnotnull(s_county#29) AND isnotnull(s_state#30)) -(43) ColumnarToRow [codegen id : 4] -Input [2]: [s_county#29, s_state#30] - -(44) BroadcastExchange +(40) CometBroadcastExchange Input [2]: [s_county#29, s_state#30] -Arguments: HashedRelationBroadcastMode(List(input[0, string, false], input[1, string, false]),false), [plan_id=4] +Arguments: [s_county#29, s_state#30] -(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 +(41) CometBroadcastHashJoin +Left output [5]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#28] +Right output [2]: [s_county#29, s_state#30] +Arguments: [ca_county#27, ca_state#28], [s_county#29, s_state#30], Inner, BuildRight -(46) Project [codegen id : 6] -Output [3]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24] +(42) CometProject 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] +Arguments: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24], [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24] + +(43) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#31, d_month_seq#32] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] +ReadSchema: struct + +(44) CometFilter +Input [2]: [d_date_sk#31, d_month_seq#32] +Condition : (((isnotnull(d_month_seq#32) AND (d_month_seq#32 >= ReusedSubquery Subquery scalar-subquery#33, [id=#34])) AND (d_month_seq#32 <= ReusedSubquery Subquery scalar-subquery#35, [id=#36])) AND isnotnull(d_date_sk#31)) + +(45) CometProject +Input [2]: [d_date_sk#31, d_month_seq#32] +Arguments: [d_date_sk#31], [d_date_sk#31] -(47) ReusedExchange [Reuses operator id: 66] -Output [1]: [d_date_sk#31] +(46) CometBroadcastExchange +Input [1]: [d_date_sk#31] +Arguments: [d_date_sk#31] -(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 +(47) CometBroadcastHashJoin +Left output [3]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24] +Right output [1]: [d_date_sk#31] +Arguments: [ss_sold_date_sk#24], [d_date_sk#31], Inner, BuildRight -(49) Project [codegen id : 6] -Output [2]: [c_customer_sk#20, ss_ext_sales_price#23] +(48) CometProject Input [4]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, d_date_sk#31] +Arguments: [c_customer_sk#20, ss_ext_sales_price#23], [c_customer_sk#20, ss_ext_sales_price#23] -(50) HashAggregate [codegen id : 6] +(49) CometHashAggregate 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] +(50) CometColumnarExchange +Input [2]: [c_customer_sk#20, sum#37] +Arguments: hashpartitioning(c_customer_sk#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(52) HashAggregate [codegen id : 7] -Input [2]: [c_customer_sk#20, sum#33] +(51) CometHashAggregate +Input [2]: [c_customer_sk#20, sum#37] 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] +(52) CometHashAggregate +Input [1]: [segment#38] +Keys [1]: [segment#38] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#36] -Results [2]: [segment#35, count#37] -(54) Exchange -Input [2]: [segment#35, count#37] -Arguments: hashpartitioning(segment#35, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(53) CometColumnarExchange +Input [2]: [segment#38, count#39] +Arguments: hashpartitioning(segment#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(55) HashAggregate [codegen id : 8] -Input [2]: [segment#35, count#37] -Keys [1]: [segment#35] +(54) CometHashAggregate +Input [2]: [segment#38, count#39] +Keys [1]: [segment#38] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#38] -Results [3]: [segment#35, count(1)#38 AS num_customers#39, (segment#35 * 50) AS segment_base#40] -(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] +(55) CometTakeOrderedAndProject +Input [3]: [segment#38, num_customers#40, segment_base#41] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[segment#38 ASC NULLS FIRST,num_customers#40 ASC NULLS FIRST], output=[segment#38,num_customers#40,segment_base#41]), [segment#38, num_customers#40, segment_base#41], 100, [segment#38 ASC NULLS FIRST, num_customers#40 ASC NULLS FIRST], [segment#38, num_customers#40, segment_base#41] + +(56) ColumnarToRow [codegen id : 1] +Input [3]: [segment#38, num_customers#40, segment_base#41] ===== Subqueries ===== @@ -360,11 +353,11 @@ Input [1]: [d_date_sk#17] (61) BroadcastExchange Input [1]: [d_date_sk#17] -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=4] Subquery:2 Hosting operator id = 4 Hosting Expression = ws_sold_date_sk#10 IN dynamicpruning#4 -Subquery:3 Hosting operator id = 29 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#25 +Subquery:3 Hosting operator id = 28 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#25 BroadcastExchange (66) +- * ColumnarToRow (65) +- CometProject (64) @@ -373,18 +366,18 @@ BroadcastExchange (66) (62) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#31, d_month_seq#41] +Output [2]: [d_date_sk#31, d_month_seq#32] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct (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)) +Input [2]: [d_date_sk#31, d_month_seq#32] +Condition : (((isnotnull(d_month_seq#32) AND (d_month_seq#32 >= Subquery scalar-subquery#33, [id=#34])) AND (d_month_seq#32 <= Subquery scalar-subquery#35, [id=#36])) AND isnotnull(d_date_sk#31)) (64) CometProject -Input [2]: [d_date_sk#31, d_month_seq#41] +Input [2]: [d_date_sk#31, d_month_seq#32] Arguments: [d_date_sk#31], [d_date_sk#31] (65) ColumnarToRow [codegen id : 1] @@ -392,12 +385,12 @@ Input [1]: [d_date_sk#31] (66) BroadcastExchange Input [1]: [d_date_sk#31] -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:4 Hosting operator id = 63 Hosting Expression = Subquery scalar-subquery#42, [id=#43] -* HashAggregate (73) -+- Exchange (72) - +- * ColumnarToRow (71) +Subquery:4 Hosting operator id = 63 Hosting Expression = Subquery scalar-subquery#33, [id=#34] +* ColumnarToRow (73) ++- CometHashAggregate (72) + +- CometColumnarExchange (71) +- CometHashAggregate (70) +- CometProject (69) +- CometFilter (68) @@ -405,43 +398,41 @@ Subquery:4 Hosting operator id = 63 Hosting Expression = Subquery scalar-subquer (67) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#46, d_year#47, d_moy#48] +Output [3]: [d_month_seq#42, d_year#43, d_moy#44] 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 (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)) +Input [3]: [d_month_seq#42, d_year#43, d_moy#44] +Condition : (((isnotnull(d_year#43) AND isnotnull(d_moy#44)) AND (d_year#43 = 1998)) AND (d_moy#44 = 12)) (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] +Input [3]: [d_month_seq#42, d_year#43, d_moy#44] +Arguments: [(d_month_seq + 1)#45], [(d_month_seq#42 + 1) AS (d_month_seq + 1)#45] (70) CometHashAggregate -Input [1]: [(d_month_seq + 1)#49] -Keys [1]: [(d_month_seq + 1)#49] +Input [1]: [(d_month_seq + 1)#45] +Keys [1]: [(d_month_seq + 1)#45] Functions: [] -(71) ColumnarToRow [codegen id : 1] -Input [1]: [(d_month_seq + 1)#49] +(71) CometColumnarExchange +Input [1]: [(d_month_seq + 1)#45] +Arguments: hashpartitioning((d_month_seq + 1)#45, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(72) Exchange -Input [1]: [(d_month_seq + 1)#49] -Arguments: hashpartitioning((d_month_seq + 1)#49, 5), ENSURE_REQUIREMENTS, [plan_id=9] - -(73) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 1)#49] -Keys [1]: [(d_month_seq + 1)#49] +(72) CometHashAggregate +Input [1]: [(d_month_seq + 1)#45] +Keys [1]: [(d_month_seq + 1)#45] Functions: [] -Aggregate Attributes: [] -Results [1]: [(d_month_seq + 1)#49] -Subquery:5 Hosting operator id = 63 Hosting Expression = Subquery scalar-subquery#44, [id=#45] -* HashAggregate (80) -+- Exchange (79) - +- * ColumnarToRow (78) +(73) ColumnarToRow [codegen id : 1] +Input [1]: [(d_month_seq + 1)#45] + +Subquery:5 Hosting operator id = 63 Hosting Expression = Subquery scalar-subquery#35, [id=#36] +* ColumnarToRow (80) ++- CometHashAggregate (79) + +- CometColumnarExchange (78) +- CometHashAggregate (77) +- CometProject (76) +- CometFilter (75) @@ -449,37 +440,39 @@ Subquery:5 Hosting operator id = 63 Hosting Expression = Subquery scalar-subquer (74) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#50, d_year#51, d_moy#52] +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 (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)) +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)) (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] +Input [3]: [d_month_seq#46, d_year#47, d_moy#48] +Arguments: [(d_month_seq + 3)#49], [(d_month_seq#46 + 3) AS (d_month_seq + 3)#49] (77) CometHashAggregate -Input [1]: [(d_month_seq + 3)#53] -Keys [1]: [(d_month_seq + 3)#53] +Input [1]: [(d_month_seq + 3)#49] +Keys [1]: [(d_month_seq + 3)#49] Functions: [] -(78) ColumnarToRow [codegen id : 1] -Input [1]: [(d_month_seq + 3)#53] +(78) CometColumnarExchange +Input [1]: [(d_month_seq + 3)#49] +Arguments: hashpartitioning((d_month_seq + 3)#49, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(79) Exchange -Input [1]: [(d_month_seq + 3)#53] -Arguments: hashpartitioning((d_month_seq + 3)#53, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(80) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 3)#53] -Keys [1]: [(d_month_seq + 3)#53] +(79) CometHashAggregate +Input [1]: [(d_month_seq + 3)#49] +Keys [1]: [(d_month_seq + 3)#49] Functions: [] -Aggregate Attributes: [] -Results [1]: [(d_month_seq + 3)#53] + +(80) ColumnarToRow [codegen id : 1] +Input [1]: [(d_month_seq + 3)#49] + +Subquery:6 Hosting operator id = 44 Hosting Expression = ReusedSubquery Subquery scalar-subquery#33, [id=#34] + +Subquery:7 Hosting operator id = 44 Hosting Expression = ReusedSubquery Subquery scalar-subquery#35, [id=#36] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/simplified.txt index 2193d875c..d3a89a6c6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/simplified.txt @@ -1,115 +1,97 @@ -TakeOrderedAndProject [segment,num_customers,segment_base] - WholeStageCodegen (8) - HashAggregate [segment,count] [count(1),num_customers,segment_base,count] - InputAdapter - Exchange [segment] #1 - 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 (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] - Project [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [ca_county,ca_state,s_county,s_state] - Project [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - HashAggregate [c_customer_sk,c_current_addr_sk] - InputAdapter - Exchange [c_customer_sk,c_current_addr_sk] #3 +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [segment,num_customers,segment_base] + CometHashAggregate [segment,num_customers,segment_base,count,count(1)] + CometColumnarExchange [segment] #1 + CometHashAggregate [segment,count] + CometHashAggregate [segment,c_customer_sk,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange [c_customer_sk] #2 + CometHashAggregate [c_customer_sk,sum,ss_ext_sales_price] + CometProject [c_customer_sk,ss_ext_sales_price] + CometBroadcastHashJoin [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometProject [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state,s_county,s_state] + CometProject [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state] + CometBroadcastHashJoin [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk,ca_address_sk,ca_county,ca_state] + CometProject [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_addr_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + CometHashAggregate [c_customer_sk,c_current_addr_sk] + CometColumnarExchange [c_customer_sk,c_current_addr_sk] #3 + CometHashAggregate [c_customer_sk,c_current_addr_sk] + CometProject [c_customer_sk,c_current_addr_sk] + CometBroadcastHashJoin [customer_sk,c_customer_sk,c_current_addr_sk] + CometProject [customer_sk] + CometBroadcastHashJoin [sold_date_sk,customer_sk,d_date_sk] + CometProject [sold_date_sk,customer_sk] + CometBroadcastHashJoin [sold_date_sk,customer_sk,item_sk,i_item_sk] + CometUnion [sold_date_sk,customer_sk,item_sk] + CometProject [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] [sold_date_sk,customer_sk,item_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [i_item_sk] #5 + CometProject [i_item_sk] + CometFilter [i_item_sk,i_class,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #7 + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] #8 + CometFilter [ss_customer_sk,ss_ext_sales_price,ss_sold_date_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 #9 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,c_current_addr_sk] - CometProject [customer_sk] - CometBroadcastHashJoin [sold_date_sk,customer_sk,d_date_sk] - CometProject [sold_date_sk,customer_sk] - CometBroadcastHashJoin [sold_date_sk,customer_sk,item_sk,i_item_sk] - CometUnion [sold_date_sk,customer_sk,item_sk] - CometProject [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [i_item_sk] #5 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_class,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #7 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk,ss_ext_sales_price,ss_sold_date_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 #9 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + Subquery #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #3 - WholeStageCodegen (2) - HashAggregate [(d_month_seq + 1)] - InputAdapter - Exchange [(d_month_seq + 1)] #10 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 1)] - CometProject [d_month_seq] [(d_month_seq + 1)] - CometFilter [d_month_seq,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - Subquery #4 - WholeStageCodegen (2) - HashAggregate [(d_month_seq + 3)] - InputAdapter - Exchange [(d_month_seq + 3)] #11 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 3)] - CometProject [d_month_seq] [(d_month_seq + 3)] - CometFilter [d_month_seq,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 - 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 #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] #9 + CometHashAggregate [(d_month_seq + 1)] + CometColumnarExchange [(d_month_seq + 1)] #10 + CometHashAggregate [(d_month_seq + 1)] + CometProject [d_month_seq] [(d_month_seq + 1)] + CometFilter [d_month_seq,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + Subquery #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [(d_month_seq + 3)] + CometColumnarExchange [(d_month_seq + 3)] #11 + CometHashAggregate [(d_month_seq + 3)] + CometProject [d_month_seq] [(d_month_seq + 3)] + CometFilter [d_month_seq,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 [ca_address_sk,ca_county,ca_state] #12 + CometFilter [ca_address_sk,ca_county,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state] + CometBroadcastExchange [s_county,s_state] #13 + CometFilter [s_county,s_state] + CometScan parquet spark_catalog.default.store [s_county,s_state] + CometBroadcastExchange [d_date_sk] #14 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + ReusedSubquery [(d_month_seq + 1)] #3 + ReusedSubquery [(d_month_seq + 3)] #4 + 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-spark3_5/q55/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/explain.txt index 00b14b96b..ee250ed9b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (19) -+- * HashAggregate (18) - +- Exchange (17) - +- * ColumnarToRow (16) +* ColumnarToRow (19) ++- CometTakeOrderedAndProject (18) + +- CometHashAggregate (17) + +- CometColumnarExchange (16) +- CometHashAggregate (15) +- CometProject (14) +- CometBroadcastHashJoin (13) @@ -93,21 +93,19 @@ 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))] -(16) ColumnarToRow [codegen id : 1] +(16) CometColumnarExchange Input [3]: [i_brand#9, i_brand_id#8, sum#11] +Arguments: hashpartitioning(i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(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] - -(18) HashAggregate [codegen id : 2] +(17) CometHashAggregate 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))#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] -(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] +(18) CometTakeOrderedAndProject +Input [3]: [brand_id#12, brand#13, ext_price#14] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#14 DESC NULLS LAST,brand_id#12 ASC NULLS FIRST], output=[brand_id#12,brand#13,ext_price#14]), [brand_id#12, brand#13, ext_price#14], 100, [ext_price#14 DESC NULLS LAST, brand_id#12 ASC NULLS FIRST], [brand_id#12, brand#13, ext_price#14] + +(19) ColumnarToRow [codegen id : 1] +Input [3]: [brand_id#12, brand#13, ext_price#14] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/simplified.txt index fc38884c1..45200509f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/simplified.txt @@ -1,23 +1,21 @@ -TakeOrderedAndProject [ext_price,brand_id,brand] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_brand,i_brand_id,sum,ss_ext_sales_price] - CometProject [ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] - CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [brand_id,brand,ext_price] + CometHashAggregate [brand_id,brand,ext_price,i_brand,i_brand_id,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange [i_brand,i_brand_id] #1 + CometHashAggregate [i_brand,i_brand_id,sum,ss_ext_sales_price] + CometProject [ss_ext_sales_price,i_brand_id,i_brand] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 + CometProject [i_item_sk,i_brand_id,i_brand] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] + 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-spark3_5/q56/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/explain.txt index 74977244e..8061fb91e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/explain.txt @@ -1,67 +1,65 @@ == Physical Plan == -TakeOrderedAndProject (63) -+- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- Union (59) - :- * HashAggregate (28) - : +- Exchange (27) - : +- * 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) - : +- * 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) - +- * 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) +* ColumnarToRow (61) ++- CometTakeOrderedAndProject (60) + +- CometHashAggregate (59) + +- CometColumnarExchange (58) + +- CometHashAggregate (57) + +- CometUnion (56) + :- CometHashAggregate (27) + : +- CometColumnarExchange (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) + :- CometHashAggregate (41) + : +- CometColumnarExchange (40) + : +- CometHashAggregate (39) + : +- CometProject (38) + : +- CometBroadcastHashJoin (37) + : :- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometProject (32) + : : : +- CometBroadcastHashJoin (31) + : : : :- CometFilter (29) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (28) + : : : +- ReusedExchange (30) + : : +- ReusedExchange (33) + : +- ReusedExchange (36) + +- CometHashAggregate (55) + +- CometColumnarExchange (54) + +- CometHashAggregate (53) + +- CometProject (52) + +- CometBroadcastHashJoin (51) + :- CometProject (49) + : +- CometBroadcastHashJoin (48) + : :- CometProject (46) + : : +- CometBroadcastHashJoin (45) + : : :- CometFilter (43) + : : : +- CometScan parquet spark_catalog.default.web_sales (42) + : : +- ReusedExchange (44) + : +- ReusedExchange (47) + +- ReusedExchange (50) (1) Scan parquet spark_catalog.default.store_sales @@ -185,212 +183,199 @@ 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))] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarExchange Input [2]: [i_item_id#12, sum#15] +Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(27) Exchange -Input [2]: [i_item_id#12, sum#15] -Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=1] - -(28) HashAggregate [codegen id : 2] +(27) CometHashAggregate 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))#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#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] +(28) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_bill_addr_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#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#22)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#20)] PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(30) CometFilter -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)) +(29) CometFilter +Input [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] +Condition : (isnotnull(cs_bill_addr_sk#16) AND isnotnull(cs_item_sk#17)) -(31) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#23] +(30) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#21] -(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 +(31) CometBroadcastHashJoin +Left output [4]: [cs_bill_addr_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 -(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] +(32) CometProject +Input [5]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19, d_date_sk#21] +Arguments: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18], [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18] -(34) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#24] +(33) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#22] -(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 +(34) CometBroadcastHashJoin +Left output [3]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18] +Right output [1]: [ca_address_sk#22] +Arguments: [cs_bill_addr_sk#16], [ca_address_sk#22], 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] +(35) CometProject +Input [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, ca_address_sk#22] +Arguments: [cs_item_sk#17, cs_ext_sales_price#18], [cs_item_sk#17, cs_ext_sales_price#18] -(37) ReusedExchange [Reuses operator id: 22] -Output [2]: [i_item_sk#25, i_item_id#26] +(36) ReusedExchange [Reuses operator id: 22] +Output [2]: [i_item_sk#23, i_item_id#24] -(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 +(37) CometBroadcastHashJoin +Left output [2]: [cs_item_sk#17, cs_ext_sales_price#18] +Right output [2]: [i_item_sk#23, i_item_id#24] +Arguments: [cs_item_sk#17], [i_item_sk#23], 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] +(38) CometProject +Input [4]: [cs_item_sk#17, cs_ext_sales_price#18, i_item_sk#23, i_item_id#24] +Arguments: [cs_ext_sales_price#18, i_item_id#24], [cs_ext_sales_price#18, i_item_id#24] -(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))] +(39) CometHashAggregate +Input [2]: [cs_ext_sales_price#18, i_item_id#24] +Keys [1]: [i_item_id#24] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#18))] -(41) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_id#26, sum#27] +(40) CometColumnarExchange +Input [2]: [i_item_id#24, sum#25] +Arguments: hashpartitioning(i_item_id#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(42) Exchange -Input [2]: [i_item_id#26, sum#27] -Arguments: hashpartitioning(i_item_id#26, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(41) CometHashAggregate +Input [2]: [i_item_id#24, sum#25] +Keys [1]: [i_item_id#24] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#18))] -(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] +(42) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] 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#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#30)] 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)) +(43) CometFilter +Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] +Condition : (isnotnull(ws_bill_addr_sk#27) AND isnotnull(ws_item_sk#26)) -(46) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#35] +(44) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#31] -(47) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] -Right output [1]: [d_date_sk#35] -Arguments: [ws_sold_date_sk#33], [d_date_sk#35], Inner, BuildRight +(45) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] +Right output [1]: [d_date_sk#31] +Arguments: [ws_sold_date_sk#29], [d_date_sk#31], Inner, BuildRight -(48) CometProject -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] -Arguments: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32], [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] +(46) CometProject +Input [5]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29, d_date_sk#31] +Arguments: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28], [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28] -(49) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#36] +(47) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#32] -(50) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] -Right output [1]: [ca_address_sk#36] -Arguments: [ws_bill_addr_sk#31], [ca_address_sk#36], Inner, BuildRight +(48) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28] +Right output [1]: [ca_address_sk#32] +Arguments: [ws_bill_addr_sk#27], [ca_address_sk#32], Inner, BuildRight -(51) CometProject -Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ca_address_sk#36] -Arguments: [ws_item_sk#30, ws_ext_sales_price#32], [ws_item_sk#30, ws_ext_sales_price#32] +(49) CometProject +Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ca_address_sk#32] +Arguments: [ws_item_sk#26, ws_ext_sales_price#28], [ws_item_sk#26, ws_ext_sales_price#28] -(52) ReusedExchange [Reuses operator id: 22] -Output [2]: [i_item_sk#37, i_item_id#38] +(50) ReusedExchange [Reuses operator id: 22] +Output [2]: [i_item_sk#33, i_item_id#34] -(53) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_ext_sales_price#32] -Right output [2]: [i_item_sk#37, i_item_id#38] -Arguments: [ws_item_sk#30], [i_item_sk#37], Inner, BuildRight +(51) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#26, ws_ext_sales_price#28] +Right output [2]: [i_item_sk#33, i_item_id#34] +Arguments: [ws_item_sk#26], [i_item_sk#33], Inner, BuildRight -(54) CometProject -Input [4]: [ws_item_sk#30, ws_ext_sales_price#32, i_item_sk#37, i_item_id#38] -Arguments: [ws_ext_sales_price#32, i_item_id#38], [ws_ext_sales_price#32, i_item_id#38] +(52) CometProject +Input [4]: [ws_item_sk#26, ws_ext_sales_price#28, i_item_sk#33, i_item_id#34] +Arguments: [ws_ext_sales_price#28, i_item_id#34], [ws_ext_sales_price#28, i_item_id#34] -(55) CometHashAggregate -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))] +(53) CometHashAggregate +Input [2]: [ws_ext_sales_price#28, i_item_id#34] +Keys [1]: [i_item_id#34] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#28))] -(56) ColumnarToRow [codegen id : 5] -Input [2]: [i_item_id#38, sum#39] +(54) CometColumnarExchange +Input [2]: [i_item_id#34, sum#35] +Arguments: hashpartitioning(i_item_id#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(57) Exchange -Input [2]: [i_item_id#38, sum#39] -Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(58) HashAggregate [codegen id : 6] -Input [2]: [i_item_id#38, sum#39] -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))#40] -Results [2]: [i_item_id#38, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#32))#40,17,2) AS total_sales#41] +(55) CometHashAggregate +Input [2]: [i_item_id#34, sum#35] +Keys [1]: [i_item_id#34] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#28))] -(59) Union +(56) CometUnion +Child 0 Input [2]: [i_item_id#12, total_sales#36] +Child 1 Input [2]: [i_item_id#24, total_sales#37] +Child 2 Input [2]: [i_item_id#34, total_sales#38] -(60) HashAggregate [codegen id : 7] -Input [2]: [i_item_id#12, total_sales#17] +(57) CometHashAggregate +Input [2]: [i_item_id#12, total_sales#36] Keys [1]: [i_item_id#12] -Functions [1]: [partial_sum(total_sales#17)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [3]: [i_item_id#12, sum#44, isEmpty#45] +Functions [1]: [partial_sum(total_sales#36)] -(61) Exchange -Input [3]: [i_item_id#12, sum#44, isEmpty#45] -Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(58) CometColumnarExchange +Input [3]: [i_item_id#12, sum#39, isEmpty#40] +Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(62) HashAggregate [codegen id : 8] -Input [3]: [i_item_id#12, sum#44, isEmpty#45] +(59) CometHashAggregate +Input [3]: [i_item_id#12, sum#39, isEmpty#40] Keys [1]: [i_item_id#12] -Functions [1]: [sum(total_sales#17)] -Aggregate Attributes [1]: [sum(total_sales#17)#46] -Results [2]: [i_item_id#12, sum(total_sales#17)#46 AS total_sales#47] +Functions [1]: [sum(total_sales#36)] + +(60) CometTakeOrderedAndProject +Input [2]: [i_item_id#12, total_sales#41] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_sales#41 ASC NULLS FIRST], output=[i_item_id#12,total_sales#41]), [i_item_id#12, total_sales#41], 100, [total_sales#41 ASC NULLS FIRST], [i_item_id#12, total_sales#41] -(63) TakeOrderedAndProject -Input [2]: [i_item_id#12, total_sales#47] -Arguments: 100, [total_sales#47 ASC NULLS FIRST], [i_item_id#12, total_sales#47] +(61) ColumnarToRow [codegen id : 1] +Input [2]: [i_item_id#12, total_sales#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (68) -+- * ColumnarToRow (67) - +- CometProject (66) - +- CometFilter (65) - +- CometScan parquet spark_catalog.default.date_dim (64) +BroadcastExchange (66) ++- * ColumnarToRow (65) + +- CometProject (64) + +- CometFilter (63) + +- CometScan parquet spark_catalog.default.date_dim (62) -(64) Scan parquet spark_catalog.default.date_dim +(62) Scan parquet spark_catalog.default.date_dim 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 +(63) 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)) -(66) CometProject +(64) CometProject 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] +(65) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(68) BroadcastExchange +(66) BroadcastExchange Input [1]: [d_date_sk#6] 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#21 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 28 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/simplified.txt index 0b48046fd..4bb76eecc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/simplified.txt @@ -1,89 +1,73 @@ -TakeOrderedAndProject [total_sales,i_item_id] - WholeStageCodegen (8) - HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - InputAdapter - Exchange [i_item_id] #1 - WholeStageCodegen (7) - HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (2) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_item_id] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,sum,ss_ext_sales_price] - CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,total_sales] + CometHashAggregate [i_item_id,total_sales,sum,isEmpty,sum(total_sales)] + CometColumnarExchange [i_item_id] #1 + CometHashAggregate [i_item_id,sum,isEmpty,total_sales] + CometUnion [i_item_id,total_sales] + CometHashAggregate [i_item_id,total_sales,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange [i_item_id] #2 + CometHashAggregate [i_item_id,sum,ss_ext_sales_price] + CometProject [ss_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ca_address_sk] #5 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [i_item_id] #7 - CometProject [i_item_id] - CometFilter [i_item_id,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] #8 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,sum,cs_ext_sales_price] - CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] - CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] - CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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 - 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] #9 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,sum,ws_ext_sales_price] - CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] - CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] - CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ca_address_sk] #5 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange [i_item_id] #7 + CometProject [i_item_id] + CometFilter [i_item_id,i_color] + CometScan parquet spark_catalog.default.item [i_item_id,i_color] + CometHashAggregate [i_item_id,total_sales,sum,sum(UnscaledValue(cs_ext_sales_price))] + CometColumnarExchange [i_item_id] #8 + CometHashAggregate [i_item_id,sum,cs_ext_sales_price] + CometProject [cs_ext_sales_price,i_item_id] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] + CometProject [cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] + CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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 + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #6 + CometHashAggregate [i_item_id,total_sales,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometColumnarExchange [i_item_id] #9 + CometHashAggregate [i_item_id,sum,ws_ext_sales_price] + CometProject [ws_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] + CometProject [ws_item_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] + CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 + 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-spark3_5/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/explain.txt index 197603ca3..b5372cffc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/explain.txt @@ -2,18 +2,18 @@ TakeOrderedAndProject (45) +- * Project (44) +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) + :- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) : :- * Project (28) : : +- * Filter (27) : : +- Window (26) : : +- * Filter (25) : : +- Window (24) - : : +- * Sort (23) - : : +- Exchange (22) - : : +- * HashAggregate (21) - : : +- Exchange (20) - : : +- * ColumnarToRow (19) + : : +- * ColumnarToRow (23) + : : +- CometSort (22) + : : +- CometColumnarExchange (21) + : : +- CometHashAggregate (20) + : : +- CometColumnarExchange (19) : : +- CometHashAggregate (18) : : +- CometProject (17) : : +- CometBroadcastHashJoin (16) @@ -32,18 +32,18 @@ TakeOrderedAndProject (45) : : +- CometBroadcastExchange (15) : : +- CometFilter (14) : : +- CometScan parquet spark_catalog.default.call_center (13) - : +- BroadcastExchange (35) - : +- * Project (34) - : +- Window (33) - : +- * Sort (32) - : +- Exchange (31) - : +- * HashAggregate (30) - : +- ReusedExchange (29) + : +- BroadcastExchange (34) + : +- * Project (33) + : +- Window (32) + : +- * ColumnarToRow (31) + : +- CometSort (30) + : +- ReusedExchange (29) +- BroadcastExchange (42) +- * Project (41) +- Window (40) - +- * Sort (39) - +- ReusedExchange (38) + +- * ColumnarToRow (39) + +- CometSort (38) + +- ReusedExchange (37) (1) Scan parquet spark_catalog.default.item @@ -135,120 +135,113 @@ Input [6]: [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_n 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))] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(20) Exchange -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 : 2] +(20) CometHashAggregate 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))#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#16, _w0#17] -Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(21) CometColumnarExchange +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16] +Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(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 +(22) CometSort +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16] +Arguments: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16], [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] + +(23) ColumnarToRow [codegen id : 1] +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16] (24) Window -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] +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16] +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#17], [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 : 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] +(25) Filter [codegen id : 2] +Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16, rn#17] 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#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 : 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 : 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#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum#25] - -(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#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 : 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#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 : 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#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 : 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)] +Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16, rn#17] +Arguments: [avg(_w0#16) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#18], [i_category#3, i_brand#2, cc_name#13, d_year#10] + +(27) Filter [codegen id : 7] +Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16, rn#17, avg_monthly_sales#18] +Condition : ((isnotnull(avg_monthly_sales#18) AND (avg_monthly_sales#18 > 0.000000)) AND CASE WHEN (avg_monthly_sales#18 > 0.000000) THEN ((abs((sum_sales#15 - avg_monthly_sales#18)) / avg_monthly_sales#18) > 0.1000000000000000) END) + +(28) Project [codegen id : 7] +Output [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, avg_monthly_sales#18, rn#17] +Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16, rn#17, avg_monthly_sales#18] + +(29) ReusedExchange [Reuses operator id: 21] +Output [6]: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15] + +(30) CometSort +Input [6]: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15] +Arguments: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15], [i_category#19 ASC NULLS FIRST, i_brand#20 ASC NULLS FIRST, cc_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] + +(31) ColumnarToRow [codegen id : 3] +Input [6]: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15] + +(32) Window +Input [6]: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15] +Arguments: [rank(d_year#22, d_moy#23) windowspecdefinition(i_category#19, i_brand#20, cc_name#21, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#24], [i_category#19, i_brand#20, cc_name#21], [d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] + +(33) Project [codegen id : 4] +Output [5]: [i_category#19, i_brand#20, cc_name#21, sum_sales#15 AS sum_sales#25, rn#24] +Input [7]: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15, rn#24] + +(34) BroadcastExchange +Input [5]: [i_category#19, i_brand#20, cc_name#21, sum_sales#25, rn#24] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=3] + +(35) BroadcastHashJoin [codegen id : 7] +Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#17] +Right keys [4]: [i_category#19, i_brand#20, cc_name#21, (rn#24 + 1)] Join type: Inner Join condition: None -(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] +(36) Project [codegen id : 7] +Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, avg_monthly_sales#18, rn#17, sum_sales#25] +Input [13]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, avg_monthly_sales#18, rn#17, i_category#19, i_brand#20, cc_name#21, sum_sales#25, rn#24] + +(37) ReusedExchange [Reuses operator id: 21] +Output [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#15] -(38) ReusedExchange [Reuses operator id: 31] -Output [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#16] +(38) CometSort +Input [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#15] +Arguments: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#15], [i_category#26 ASC NULLS FIRST, i_brand#27 ASC NULLS FIRST, cc_name#28 ASC NULLS FIRST, d_year#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST] -(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 +(39) ColumnarToRow [codegen id : 5] +Input [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#15] (40) Window -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] +Input [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#15] +Arguments: [rank(d_year#29, d_moy#30) windowspecdefinition(i_category#26, i_brand#27, cc_name#28, d_year#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#31], [i_category#26, i_brand#27, cc_name#28], [d_year#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST] -(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] +(41) Project [codegen id : 6] +Output [5]: [i_category#26, i_brand#27, cc_name#28, sum_sales#15 AS sum_sales#32, rn#31] +Input [7]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#15, rn#31] (42) BroadcastExchange -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] +Input [5]: [i_category#26, i_brand#27, cc_name#28, sum_sales#32, rn#31] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=4] -(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)] +(43) BroadcastHashJoin [codegen id : 7] +Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#17] +Right keys [4]: [i_category#26, i_brand#27, cc_name#28, (rn#31 - 1)] Join type: Inner Join condition: None -(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] +(44) Project [codegen id : 7] +Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#18, sum_sales#15, sum_sales#25 AS psum#33, sum_sales#32 AS nsum#34] +Input [14]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, avg_monthly_sales#18, rn#17, sum_sales#25, i_category#26, i_brand#27, cc_name#28, sum_sales#32, rn#31] (45) TakeOrderedAndProject -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] +Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#18, sum_sales#15, psum#33, nsum#34] +Arguments: 100, [(sum_sales#15 - avg_monthly_sales#18) 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#18, sum_sales#15, psum#33, nsum#34] ===== Subqueries ===== @@ -275,6 +268,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=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/simplified.txt index fb2b09b9d..454d12fae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/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 (13) + WholeStageCodegen (7) 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,67 +8,61 @@ 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 (4) + WholeStageCodegen (2) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (3) - Sort [i_category,i_brand,cc_name,d_year,d_moy] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [i_category,i_brand,cc_name] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum,cs_sales_price] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometFilter [i_item_sk,i_brand,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 - CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,cc_name] #1 + CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum,sum(UnscaledValue(cs_sales_price))] + CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum,cs_sales_price] + CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_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_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 + CometFilter [cc_call_center_sk,cc_name] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter BroadcastExchange #7 - WholeStageCodegen (8) + WholeStageCodegen (4) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (7) - Sort [i_category,i_brand,cc_name,d_year,d_moy] + WholeStageCodegen (3) + ColumnarToRow InputAdapter - 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 + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #1 InputAdapter - BroadcastExchange #9 - WholeStageCodegen (12) + BroadcastExchange #8 + WholeStageCodegen (6) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (11) - Sort [i_category,i_brand,cc_name,d_year,d_moy] + WholeStageCodegen (5) + ColumnarToRow InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/explain.txt index 9d5fe57a2..06e3fb959 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/explain.txt @@ -1,57 +1,55 @@ == Physical Plan == -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) +* ColumnarToRow (51) ++- CometTakeOrderedAndProject (50) + +- CometProject (49) + +- CometBroadcastHashJoin (48) + :- CometProject (34) + : +- CometBroadcastHashJoin (33) + : :- CometFilter (19) + : : +- CometHashAggregate (18) + : : +- CometColumnarExchange (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) + : +- CometBroadcastExchange (32) + : +- CometFilter (31) + : +- CometHashAggregate (30) + : +- CometColumnarExchange (29) + : +- CometHashAggregate (28) + : +- CometProject (27) + : +- CometBroadcastHashJoin (26) + : :- CometProject (24) + : : +- CometBroadcastHashJoin (23) + : : :- CometFilter (21) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (20) + : : +- ReusedExchange (22) + : +- ReusedExchange (25) + +- CometBroadcastExchange (47) + +- CometFilter (46) + +- CometHashAggregate (45) + +- CometColumnarExchange (44) + +- CometHashAggregate (43) + +- CometProject (42) + +- CometBroadcastHashJoin (41) + :- CometProject (39) + : +- CometBroadcastHashJoin (38) + : :- CometFilter (36) + : : +- CometScan parquet spark_catalog.default.web_sales (35) + : +- ReusedExchange (37) + +- ReusedExchange (40) (1) Scan parquet spark_catalog.default.store_sales @@ -101,7 +99,7 @@ ReadSchema: struct Input [2]: [d_date_sk#7, d_date#8] Condition : isnotnull(d_date_sk#7) -(10) ReusedExchange [Reuses operator id: 59] +(10) ReusedExchange [Reuses operator id: 57] Output [1]: [d_date#9] (11) CometBroadcastHashJoin @@ -131,262 +129,248 @@ 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))] -(17) ColumnarToRow [codegen id : 1] +(17) CometColumnarExchange Input [2]: [i_item_id#6, sum#10] +Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(18) Exchange -Input [2]: [i_item_id#6, sum#10] -Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [plan_id=1] - -(19) HashAggregate [codegen id : 6] +(18) CometHashAggregate 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))#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] -(20) Filter [codegen id : 6] -Input [2]: [item_id#12, ss_item_rev#13] -Condition : isnotnull(ss_item_rev#13) +(19) CometFilter +Input [2]: [item_id#11, ss_item_rev#12] +Condition : isnotnull(ss_item_rev#12) -(21) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] +(20) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_item_sk#13, cs_ext_sales_price#14, cs_sold_date_sk#15] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] +PartitionFilters: [isnotnull(cs_sold_date_sk#15), dynamicpruningexpression(cs_sold_date_sk#15 IN dynamicpruning#16)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(22) CometFilter -Input [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] -Condition : isnotnull(cs_item_sk#14) - -(23) ReusedExchange [Reuses operator id: 5] -Output [2]: [i_item_sk#18, i_item_id#19] - -(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 - -(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] - -(26) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#20] - -(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 - -(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) - -(34) BroadcastExchange -Input [2]: [item_id#23, cs_item_rev#24] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] - -(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#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))) - -(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] - -(37) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] +(21) CometFilter +Input [3]: [cs_item_sk#13, cs_ext_sales_price#14, cs_sold_date_sk#15] +Condition : isnotnull(cs_item_sk#13) + +(22) ReusedExchange [Reuses operator id: 5] +Output [2]: [i_item_sk#17, i_item_id#18] + +(23) CometBroadcastHashJoin +Left output [3]: [cs_item_sk#13, cs_ext_sales_price#14, cs_sold_date_sk#15] +Right output [2]: [i_item_sk#17, i_item_id#18] +Arguments: [cs_item_sk#13], [i_item_sk#17], Inner, BuildRight + +(24) CometProject +Input [5]: [cs_item_sk#13, cs_ext_sales_price#14, cs_sold_date_sk#15, i_item_sk#17, i_item_id#18] +Arguments: [cs_ext_sales_price#14, cs_sold_date_sk#15, i_item_id#18], [cs_ext_sales_price#14, cs_sold_date_sk#15, i_item_id#18] + +(25) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_date_sk#19] + +(26) CometBroadcastHashJoin +Left output [3]: [cs_ext_sales_price#14, cs_sold_date_sk#15, i_item_id#18] +Right output [1]: [d_date_sk#19] +Arguments: [cs_sold_date_sk#15], [d_date_sk#19], Inner, BuildRight + +(27) CometProject +Input [4]: [cs_ext_sales_price#14, cs_sold_date_sk#15, i_item_id#18, d_date_sk#19] +Arguments: [cs_ext_sales_price#14, i_item_id#18], [cs_ext_sales_price#14, i_item_id#18] + +(28) CometHashAggregate +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))] + +(29) CometColumnarExchange +Input [2]: [i_item_id#18, sum#20] +Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(30) CometHashAggregate +Input [2]: [i_item_id#18, sum#20] +Keys [1]: [i_item_id#18] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#14))] + +(31) CometFilter +Input [2]: [item_id#21, cs_item_rev#22] +Condition : isnotnull(cs_item_rev#22) + +(32) CometBroadcastExchange +Input [2]: [item_id#21, cs_item_rev#22] +Arguments: [item_id#21, cs_item_rev#22] + +(33) CometBroadcastHashJoin +Left output [2]: [item_id#11, ss_item_rev#12] +Right output [2]: [item_id#21, cs_item_rev#22] +Arguments: [item_id#11], [item_id#21], Inner, ((((cast(ss_item_rev#12 as decimal(19,3)) >= (0.9 * cs_item_rev#22)) AND (cast(ss_item_rev#12 as decimal(20,3)) <= (1.1 * cs_item_rev#22))) AND (cast(cs_item_rev#22 as decimal(19,3)) >= (0.9 * ss_item_rev#12))) AND (cast(cs_item_rev#22 as decimal(20,3)) <= (1.1 * ss_item_rev#12))), BuildRight + +(34) CometProject +Input [4]: [item_id#11, ss_item_rev#12, item_id#21, cs_item_rev#22] +Arguments: [item_id#11, ss_item_rev#12, cs_item_rev#22], [item_id#11, ss_item_rev#12, cs_item_rev#22] + +(35) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_item_sk#23, ws_ext_sales_price#24, ws_sold_date_sk#25] 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#25), dynamicpruningexpression(ws_sold_date_sk#25 IN dynamicpruning#26)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(38) CometFilter -Input [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] -Condition : isnotnull(ws_item_sk#25) +(36) CometFilter +Input [3]: [ws_item_sk#23, ws_ext_sales_price#24, ws_sold_date_sk#25] +Condition : isnotnull(ws_item_sk#23) + +(37) ReusedExchange [Reuses operator id: 5] +Output [2]: [i_item_sk#27, i_item_id#28] -(39) ReusedExchange [Reuses operator id: 5] -Output [2]: [i_item_sk#29, i_item_id#30] +(38) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#23, ws_ext_sales_price#24, ws_sold_date_sk#25] +Right output [2]: [i_item_sk#27, i_item_id#28] +Arguments: [ws_item_sk#23], [i_item_sk#27], Inner, BuildRight -(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 +(39) CometProject +Input [5]: [ws_item_sk#23, ws_ext_sales_price#24, ws_sold_date_sk#25, i_item_sk#27, i_item_id#28] +Arguments: [ws_ext_sales_price#24, ws_sold_date_sk#25, i_item_id#28], [ws_ext_sales_price#24, ws_sold_date_sk#25, i_item_id#28] -(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] +(40) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_date_sk#29] -(42) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#31] +(41) CometBroadcastHashJoin +Left output [3]: [ws_ext_sales_price#24, ws_sold_date_sk#25, i_item_id#28] +Right output [1]: [d_date_sk#29] +Arguments: [ws_sold_date_sk#25], [d_date_sk#29], Inner, BuildRight -(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 +(42) CometProject +Input [4]: [ws_ext_sales_price#24, ws_sold_date_sk#25, i_item_id#28, d_date_sk#29] +Arguments: [ws_ext_sales_price#24, i_item_id#28], [ws_ext_sales_price#24, i_item_id#28] -(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] +(43) CometHashAggregate +Input [2]: [ws_ext_sales_price#24, i_item_id#28] +Keys [1]: [i_item_id#28] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#24))] + +(44) CometColumnarExchange +Input [2]: [i_item_id#28, sum#30] +Arguments: hashpartitioning(i_item_id#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (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))] - -(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] - -(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))#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] - -(49) Filter [codegen id : 5] -Input [2]: [item_id#34, ws_item_rev#35] -Condition : isnotnull(ws_item_rev#35) - -(50) BroadcastExchange -Input [2]: [item_id#34, ws_item_rev#35] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] - -(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#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))) - -(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] - -(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] +Input [2]: [i_item_id#28, sum#30] +Keys [1]: [i_item_id#28] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#24))] -===== Subqueries ===== +(46) CometFilter +Input [2]: [item_id#31, ws_item_rev#32] +Condition : isnotnull(ws_item_rev#32) -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -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) +(47) CometBroadcastExchange +Input [2]: [item_id#31, ws_item_rev#32] +Arguments: [item_id#31, ws_item_rev#32] +(48) CometBroadcastHashJoin +Left output [3]: [item_id#11, ss_item_rev#12, cs_item_rev#22] +Right output [2]: [item_id#31, ws_item_rev#32] +Arguments: [item_id#11], [item_id#31], Inner, ((((((((cast(ss_item_rev#12 as decimal(19,3)) >= (0.9 * ws_item_rev#32)) AND (cast(ss_item_rev#12 as decimal(20,3)) <= (1.1 * ws_item_rev#32))) AND (cast(cs_item_rev#22 as decimal(19,3)) >= (0.9 * ws_item_rev#32))) AND (cast(cs_item_rev#22 as decimal(20,3)) <= (1.1 * ws_item_rev#32))) AND (cast(ws_item_rev#32 as decimal(19,3)) >= (0.9 * ss_item_rev#12))) AND (cast(ws_item_rev#32 as decimal(20,3)) <= (1.1 * ss_item_rev#12))) AND (cast(ws_item_rev#32 as decimal(19,3)) >= (0.9 * cs_item_rev#22))) AND (cast(ws_item_rev#32 as decimal(20,3)) <= (1.1 * cs_item_rev#22))), BuildRight -(54) Scan parquet spark_catalog.default.date_dim +(49) CometProject +Input [5]: [item_id#11, ss_item_rev#12, cs_item_rev#22, item_id#31, ws_item_rev#32] +Arguments: [item_id#11, ss_item_rev#12, ss_dev#33, cs_item_rev#22, cs_dev#34, ws_item_rev#32, ws_dev#35, average#36], [item_id#11, ss_item_rev#12, (((ss_item_rev#12 / ((ss_item_rev#12 + cs_item_rev#22) + ws_item_rev#32)) / 3) * 100) AS ss_dev#33, cs_item_rev#22, (((cs_item_rev#22 / ((ss_item_rev#12 + cs_item_rev#22) + ws_item_rev#32)) / 3) * 100) AS cs_dev#34, ws_item_rev#32, (((ws_item_rev#32 / ((ss_item_rev#12 + cs_item_rev#22) + ws_item_rev#32)) / 3) * 100) AS ws_dev#35, (((ss_item_rev#12 + cs_item_rev#22) + ws_item_rev#32) / 3) AS average#36] + +(50) CometTakeOrderedAndProject +Input [8]: [item_id#11, ss_item_rev#12, ss_dev#33, cs_item_rev#22, cs_dev#34, ws_item_rev#32, ws_dev#35, average#36] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_id#11 ASC NULLS FIRST,ss_item_rev#12 ASC NULLS FIRST], output=[item_id#11,ss_item_rev#12,ss_dev#33,cs_item_rev#22,cs_dev#34,ws_item_rev#32,ws_dev#35,average#36]), [item_id#11, ss_item_rev#12, ss_dev#33, cs_item_rev#22, cs_dev#34, ws_item_rev#32, ws_dev#35, average#36], 100, [item_id#11 ASC NULLS FIRST, ss_item_rev#12 ASC NULLS FIRST], [item_id#11, ss_item_rev#12, ss_dev#33, cs_item_rev#22, cs_dev#34, ws_item_rev#32, ws_dev#35, average#36] + +(51) ColumnarToRow [codegen id : 1] +Input [8]: [item_id#11, ss_item_rev#12, ss_dev#33, cs_item_rev#22, cs_dev#34, ws_item_rev#32, ws_dev#35, average#36] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 +BroadcastExchange (61) ++- * ColumnarToRow (60) + +- CometProject (59) + +- CometBroadcastHashJoin (58) + :- CometFilter (53) + : +- CometScan parquet spark_catalog.default.date_dim (52) + +- CometBroadcastExchange (57) + +- CometProject (56) + +- CometFilter (55) + +- CometScan parquet spark_catalog.default.date_dim (54) + + +(52) 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 -(55) CometFilter +(53) CometFilter Input [2]: [d_date_sk#7, d_date#8] Condition : isnotnull(d_date_sk#7) -(56) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date#9, d_week_seq#40] +(54) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date#9, d_week_seq#37] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct -(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) CometFilter +Input [2]: [d_date#9, d_week_seq#37] +Condition : (isnotnull(d_week_seq#37) AND (d_week_seq#37 = Subquery scalar-subquery#38, [id=#39])) -(58) CometProject -Input [2]: [d_date#9, d_week_seq#40] +(56) CometProject +Input [2]: [d_date#9, d_week_seq#37] Arguments: [d_date#9], [d_date#9] -(59) CometBroadcastExchange +(57) CometBroadcastExchange Input [1]: [d_date#9] Arguments: [d_date#9] -(60) CometBroadcastHashJoin +(58) 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 -(61) CometProject +(59) CometProject Input [2]: [d_date_sk#7, d_date#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(62) ColumnarToRow [codegen id : 1] +(60) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(63) BroadcastExchange +(61) 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] -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) +Subquery:2 Hosting operator id = 55 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +* ColumnarToRow (65) ++- CometProject (64) + +- CometFilter (63) + +- CometScan parquet spark_catalog.default.date_dim (62) -(64) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date#43, d_week_seq#44] +(62) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date#40, d_week_seq#41] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] ReadSchema: struct -(65) CometFilter -Input [2]: [d_date#43, d_week_seq#44] -Condition : (isnotnull(d_date#43) AND (d_date#43 = 2000-01-03)) +(63) CometFilter +Input [2]: [d_date#40, d_week_seq#41] +Condition : (isnotnull(d_date#40) AND (d_date#40 = 2000-01-03)) -(66) CometProject -Input [2]: [d_date#43, d_week_seq#44] -Arguments: [d_week_seq#44], [d_week_seq#44] +(64) CometProject +Input [2]: [d_date#40, d_week_seq#41] +Arguments: [d_week_seq#41], [d_week_seq#41] -(67) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#44] +(65) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#41] -Subquery:3 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#4 +Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#4 -Subquery:4 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#4 +Subquery:4 Hosting operator id = 35 Hosting Expression = ws_sold_date_sk#25 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/simplified.txt index fc7c21e94..326fdcb78 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/simplified.txt @@ -1,89 +1,75 @@ -TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] - 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] - BroadcastHashJoin [item_id,item_id,ss_item_rev,cs_item_rev] - Filter [ss_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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,sum,ss_ext_sales_price] - CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,d_date_sk] - CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id] - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_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 (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] - CometFilter [d_date,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] - 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 [i_item_sk,i_item_id] #4 - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date] #3 - InputAdapter - 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] #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,sum,cs_ext_sales_price] - CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,d_date_sk] - CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id] - CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [i_item_sk,i_item_id] #4 - ReusedExchange [d_date_sk] #5 - InputAdapter - 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] #9 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,sum,ws_ext_sales_price] - CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,d_date_sk] - CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id] - CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [i_item_sk,i_item_id] #4 - ReusedExchange [d_date_sk] #5 +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] + CometProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] + CometBroadcastHashJoin [item_id,ss_item_rev,cs_item_rev,item_id,ws_item_rev] + CometProject [item_id,ss_item_rev,cs_item_rev] + CometBroadcastHashJoin [item_id,ss_item_rev,item_id,cs_item_rev] + CometFilter [item_id,ss_item_rev] + CometHashAggregate [item_id,ss_item_rev,i_item_id,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange [i_item_id] #1 + CometHashAggregate [i_item_id,sum,ss_ext_sales_price] + CometProject [ss_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,d_date_sk] + CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_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 (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date] #3 + CometProject [d_date] + CometFilter [d_date,d_week_seq] + Subquery #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_date,d_week_seq] + 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 [i_item_sk,i_item_id] #4 + CometFilter [i_item_sk,i_item_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + ReusedExchange [d_date] #3 + CometBroadcastExchange [item_id,cs_item_rev] #6 + CometFilter [item_id,cs_item_rev] + CometHashAggregate [item_id,cs_item_rev,i_item_id,sum,sum(UnscaledValue(cs_ext_sales_price))] + CometColumnarExchange [i_item_id] #7 + CometHashAggregate [i_item_id,sum,cs_ext_sales_price] + CometProject [cs_ext_sales_price,i_item_id] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,d_date_sk] + CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [i_item_sk,i_item_id] #4 + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange [item_id,ws_item_rev] #8 + CometFilter [item_id,ws_item_rev] + CometHashAggregate [item_id,ws_item_rev,i_item_id,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometColumnarExchange [i_item_id] #9 + CometHashAggregate [i_item_id,sum,ws_ext_sales_price] + CometProject [ws_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,d_date_sk] + CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + 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-spark3_5/q59/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/explain.txt index 20e2a87d4..51afdb3c7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/explain.txt @@ -1,47 +1,43 @@ == Physical Plan == -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) +* ColumnarToRow (39) ++- CometTakeOrderedAndProject (38) + +- CometProject (37) + +- CometBroadcastHashJoin (36) + :- CometProject (21) + : +- CometBroadcastHashJoin (20) + : :- CometProject (15) + : : +- CometBroadcastHashJoin (14) + : : :- CometHashAggregate (10) + : : : +- CometColumnarExchange (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) + : : +- CometBroadcastExchange (13) + : : +- CometFilter (12) + : : +- CometScan parquet spark_catalog.default.store (11) + : +- CometBroadcastExchange (19) + : +- CometProject (18) + : +- CometFilter (17) + : +- CometScan parquet spark_catalog.default.date_dim (16) + +- CometBroadcastExchange (35) + +- CometProject (34) + +- CometBroadcastHashJoin (33) + :- CometProject (28) + : +- CometBroadcastHashJoin (27) + : :- CometHashAggregate (23) + : : +- ReusedExchange (22) + : +- CometBroadcastExchange (26) + : +- CometFilter (25) + : +- CometScan parquet spark_catalog.default.store (24) + +- CometBroadcastExchange (32) + +- CometProject (31) + +- CometFilter (30) + +- CometScan parquet spark_catalog.default.date_dim (29) (1) Scan parquet spark_catalog.default.store_sales @@ -85,165 +81,144 @@ 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))] -(9) ColumnarToRow [codegen id : 1] +(9) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(11) HashAggregate [codegen id : 8] +(10) CometHashAggregate 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))#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] -(12) Scan parquet spark_catalog.default.store -Output [3]: [s_store_sk#28, s_store_id#29, s_store_name#30] +(11) Scan parquet spark_catalog.default.store +Output [3]: [s_store_sk#14, s_store_id#15, s_store_name#16] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_id)] ReadSchema: struct -(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)) - -(14) ColumnarToRow [codegen id : 2] -Input [3]: [s_store_sk#28, s_store_id#29, s_store_name#30] +(12) CometFilter +Input [3]: [s_store_sk#14, s_store_id#15, s_store_name#16] +Condition : (isnotnull(s_store_sk#14) AND isnotnull(s_store_id#15)) -(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] +(13) CometBroadcastExchange +Input [3]: [s_store_sk#14, s_store_id#15, s_store_name#16] +Arguments: [s_store_sk#14, s_store_id#15, s_store_name#16] -(16) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#28] -Join type: Inner -Join condition: None +(14) CometBroadcastHashJoin +Left output [9]: [d_week_seq#5, ss_store_sk#1, sun_sales#17, mon_sales#18, tue_sales#19, wed_sales#20, thu_sales#21, fri_sales#22, sat_sales#23] +Right output [3]: [s_store_sk#14, s_store_id#15, s_store_name#16] +Arguments: [ss_store_sk#1], [s_store_sk#14], Inner, BuildRight -(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] +(15) CometProject +Input [12]: [d_week_seq#5, ss_store_sk#1, sun_sales#17, mon_sales#18, tue_sales#19, wed_sales#20, thu_sales#21, fri_sales#22, sat_sales#23, s_store_sk#14, s_store_id#15, s_store_name#16] +Arguments: [d_week_seq#5, sun_sales#17, mon_sales#18, tue_sales#19, wed_sales#20, thu_sales#21, fri_sales#22, sat_sales#23, s_store_id#15, s_store_name#16], [d_week_seq#5, sun_sales#17, mon_sales#18, tue_sales#19, wed_sales#20, thu_sales#21, fri_sales#22, sat_sales#23, s_store_id#15, s_store_name#16] -(18) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#31, d_week_seq#32] +(16) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_month_seq#24, d_week_seq#25] 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 -(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)) +(17) CometFilter +Input [2]: [d_month_seq#24, d_week_seq#25] +Condition : (((isnotnull(d_month_seq#24) AND (d_month_seq#24 >= 1212)) AND (d_month_seq#24 <= 1223)) AND isnotnull(d_week_seq#25)) -(20) CometProject -Input [2]: [d_month_seq#31, d_week_seq#32] -Arguments: [d_week_seq#32], [d_week_seq#32] +(18) CometProject +Input [2]: [d_month_seq#24, d_week_seq#25] +Arguments: [d_week_seq#25], [d_week_seq#25] -(21) ColumnarToRow [codegen id : 3] -Input [1]: [d_week_seq#32] +(19) CometBroadcastExchange +Input [1]: [d_week_seq#25] +Arguments: [d_week_seq#25] -(22) BroadcastExchange -Input [1]: [d_week_seq#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(20) CometBroadcastHashJoin +Left output [10]: [d_week_seq#5, sun_sales#17, mon_sales#18, tue_sales#19, wed_sales#20, thu_sales#21, fri_sales#22, sat_sales#23, s_store_id#15, s_store_name#16] +Right output [1]: [d_week_seq#25] +Arguments: [d_week_seq#5], [d_week_seq#25], Inner, BuildRight -(23) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [d_week_seq#5] -Right keys [1]: [d_week_seq#32] -Join type: Inner -Join condition: None +(21) CometProject +Input [11]: [d_week_seq#5, sun_sales#17, mon_sales#18, tue_sales#19, wed_sales#20, thu_sales#21, fri_sales#22, sat_sales#23, s_store_id#15, s_store_name#16, d_week_seq#25] +Arguments: [s_store_name1#26, d_week_seq1#27, s_store_id1#28, sun_sales1#29, mon_sales1#30, tue_sales1#31, wed_sales1#32, thu_sales1#33, fri_sales1#34, sat_sales1#35], [s_store_name#16 AS s_store_name1#26, d_week_seq#5 AS d_week_seq1#27, s_store_id#15 AS s_store_id1#28, sun_sales#17 AS sun_sales1#29, mon_sales#18 AS mon_sales1#30, tue_sales#19 AS tue_sales1#31, wed_sales#20 AS wed_sales1#32, thu_sales#21 AS thu_sales1#33, fri_sales#22 AS fri_sales1#34, sat_sales#23 AS sat_sales1#35] -(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] +(22) ReusedExchange [Reuses operator id: 9] +Output [9]: [d_week_seq#36, ss_store_sk#37, sum#38, sum#39, sum#40, sum#41, sum#42, sum#43, sum#44] -(25) ReusedExchange [Reuses operator id: 10] -Output [9]: [d_week_seq#43, ss_store_sk#44, sum#45, sum#46, sum#47, sum#48, sum#49, sum#50, sum#51] +(23) CometHashAggregate +Input [9]: [d_week_seq#36, ss_store_sk#37, sum#38, sum#39, sum#40, sum#41, sum#42, sum#43, sum#44] +Keys [2]: [d_week_seq#36, ss_store_sk#37] +Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#45 = Sunday ) THEN ss_sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Monday ) THEN ss_sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Tuesday ) THEN ss_sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Wednesday) THEN ss_sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Thursday ) THEN ss_sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Friday ) THEN ss_sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Saturday ) THEN ss_sales_price#46 END))] -(26) HashAggregate [codegen id : 7] -Input [9]: [d_week_seq#43, ss_store_sk#44, sum#45, sum#46, sum#47, sum#48, sum#49, sum#50, sum#51] -Keys [2]: [d_week_seq#43, ss_store_sk#44] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#52 = Sunday ) THEN ss_sales_price#53 END)), sum(UnscaledValue(CASE WHEN (d_day_name#52 = Monday ) THEN ss_sales_price#53 END)), sum(UnscaledValue(CASE WHEN (d_day_name#52 = Tuesday ) THEN ss_sales_price#53 END)), sum(UnscaledValue(CASE WHEN (d_day_name#52 = Wednesday) THEN ss_sales_price#53 END)), sum(UnscaledValue(CASE WHEN (d_day_name#52 = Thursday ) THEN ss_sales_price#53 END)), sum(UnscaledValue(CASE WHEN (d_day_name#52 = Friday ) THEN ss_sales_price#53 END)), sum(UnscaledValue(CASE WHEN (d_day_name#52 = Saturday ) THEN ss_sales_price#53 END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#52 = Sunday ) THEN ss_sales_price#53 END))#14, sum(UnscaledValue(CASE WHEN (d_day_name#52 = Monday ) THEN ss_sales_price#53 END))#15, sum(UnscaledValue(CASE WHEN (d_day_name#52 = Tuesday ) THEN ss_sales_price#53 END))#16, sum(UnscaledValue(CASE WHEN (d_day_name#52 = Wednesday) THEN ss_sales_price#53 END))#17, sum(UnscaledValue(CASE WHEN (d_day_name#52 = Thursday ) THEN ss_sales_price#53 END))#18, sum(UnscaledValue(CASE WHEN (d_day_name#52 = Friday ) THEN ss_sales_price#53 END))#19, sum(UnscaledValue(CASE WHEN (d_day_name#52 = Saturday ) THEN ss_sales_price#53 END))#20] -Results [9]: [d_week_seq#43, ss_store_sk#44, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#52 = Sunday ) THEN ss_sales_price#53 END))#14,17,2) AS sun_sales#54, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#52 = Monday ) THEN ss_sales_price#53 END))#15,17,2) AS mon_sales#55, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#52 = Tuesday ) THEN ss_sales_price#53 END))#16,17,2) AS tue_sales#56, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#52 = Wednesday) THEN ss_sales_price#53 END))#17,17,2) AS wed_sales#57, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#52 = Thursday ) THEN ss_sales_price#53 END))#18,17,2) AS thu_sales#58, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#52 = Friday ) THEN ss_sales_price#53 END))#19,17,2) AS fri_sales#59, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#52 = Saturday ) THEN ss_sales_price#53 END))#20,17,2) AS sat_sales#60] - -(27) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#61, s_store_id#62] +(24) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#47, s_store_id#48] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_id)] ReadSchema: struct -(28) CometFilter -Input [2]: [s_store_sk#61, s_store_id#62] -Condition : (isnotnull(s_store_sk#61) AND isnotnull(s_store_id#62)) - -(29) ColumnarToRow [codegen id : 5] -Input [2]: [s_store_sk#61, s_store_id#62] +(25) CometFilter +Input [2]: [s_store_sk#47, s_store_id#48] +Condition : (isnotnull(s_store_sk#47) AND isnotnull(s_store_id#48)) -(30) BroadcastExchange -Input [2]: [s_store_sk#61, s_store_id#62] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(26) CometBroadcastExchange +Input [2]: [s_store_sk#47, s_store_id#48] +Arguments: [s_store_sk#47, s_store_id#48] -(31) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_store_sk#44] -Right keys [1]: [s_store_sk#61] -Join type: Inner -Join condition: None +(27) CometBroadcastHashJoin +Left output [9]: [d_week_seq#36, ss_store_sk#37, sun_sales#49, mon_sales#50, tue_sales#51, wed_sales#52, thu_sales#53, fri_sales#54, sat_sales#55] +Right output [2]: [s_store_sk#47, s_store_id#48] +Arguments: [ss_store_sk#37], [s_store_sk#47], Inner, BuildRight -(32) Project [codegen id : 7] -Output [9]: [d_week_seq#43, sun_sales#54, mon_sales#55, tue_sales#56, wed_sales#57, thu_sales#58, fri_sales#59, sat_sales#60, s_store_id#62] -Input [11]: [d_week_seq#43, ss_store_sk#44, sun_sales#54, mon_sales#55, tue_sales#56, wed_sales#57, thu_sales#58, fri_sales#59, sat_sales#60, s_store_sk#61, s_store_id#62] +(28) CometProject +Input [11]: [d_week_seq#36, ss_store_sk#37, sun_sales#49, mon_sales#50, tue_sales#51, wed_sales#52, thu_sales#53, fri_sales#54, sat_sales#55, s_store_sk#47, s_store_id#48] +Arguments: [d_week_seq#36, sun_sales#49, mon_sales#50, tue_sales#51, wed_sales#52, thu_sales#53, fri_sales#54, sat_sales#55, s_store_id#48], [d_week_seq#36, sun_sales#49, mon_sales#50, tue_sales#51, wed_sales#52, thu_sales#53, fri_sales#54, sat_sales#55, s_store_id#48] -(33) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#63, d_week_seq#64] +(29) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_month_seq#56, d_week_seq#57] 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 -(34) CometFilter -Input [2]: [d_month_seq#63, d_week_seq#64] -Condition : (((isnotnull(d_month_seq#63) AND (d_month_seq#63 >= 1224)) AND (d_month_seq#63 <= 1235)) AND isnotnull(d_week_seq#64)) +(30) CometFilter +Input [2]: [d_month_seq#56, d_week_seq#57] +Condition : (((isnotnull(d_month_seq#56) AND (d_month_seq#56 >= 1224)) AND (d_month_seq#56 <= 1235)) AND isnotnull(d_week_seq#57)) -(35) CometProject -Input [2]: [d_month_seq#63, d_week_seq#64] -Arguments: [d_week_seq#64], [d_week_seq#64] +(31) CometProject +Input [2]: [d_month_seq#56, d_week_seq#57] +Arguments: [d_week_seq#57], [d_week_seq#57] -(36) ColumnarToRow [codegen id : 6] -Input [1]: [d_week_seq#64] +(32) CometBroadcastExchange +Input [1]: [d_week_seq#57] +Arguments: [d_week_seq#57] -(37) BroadcastExchange -Input [1]: [d_week_seq#64] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(33) CometBroadcastHashJoin +Left output [9]: [d_week_seq#36, sun_sales#49, mon_sales#50, tue_sales#51, wed_sales#52, thu_sales#53, fri_sales#54, sat_sales#55, s_store_id#48] +Right output [1]: [d_week_seq#57] +Arguments: [d_week_seq#36], [d_week_seq#57], Inner, BuildRight -(38) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [d_week_seq#43] -Right keys [1]: [d_week_seq#64] -Join type: Inner -Join condition: None +(34) CometProject +Input [10]: [d_week_seq#36, sun_sales#49, mon_sales#50, tue_sales#51, wed_sales#52, thu_sales#53, fri_sales#54, sat_sales#55, s_store_id#48, d_week_seq#57] +Arguments: [d_week_seq2#58, s_store_id2#59, sun_sales2#60, mon_sales2#61, tue_sales2#62, wed_sales2#63, thu_sales2#64, fri_sales2#65, sat_sales2#66], [d_week_seq#36 AS d_week_seq2#58, s_store_id#48 AS s_store_id2#59, sun_sales#49 AS sun_sales2#60, mon_sales#50 AS mon_sales2#61, tue_sales#51 AS tue_sales2#62, wed_sales#52 AS wed_sales2#63, thu_sales#53 AS thu_sales2#64, fri_sales#54 AS fri_sales2#65, sat_sales#55 AS sat_sales2#66] -(39) Project [codegen id : 7] -Output [9]: [d_week_seq#43 AS d_week_seq2#65, s_store_id#62 AS s_store_id2#66, sun_sales#54 AS sun_sales2#67, mon_sales#55 AS mon_sales2#68, tue_sales#56 AS tue_sales2#69, wed_sales#57 AS wed_sales2#70, thu_sales#58 AS thu_sales2#71, fri_sales#59 AS fri_sales2#72, sat_sales#60 AS sat_sales2#73] -Input [10]: [d_week_seq#43, sun_sales#54, mon_sales#55, tue_sales#56, wed_sales#57, thu_sales#58, fri_sales#59, sat_sales#60, s_store_id#62, d_week_seq#64] +(35) CometBroadcastExchange +Input [9]: [d_week_seq2#58, s_store_id2#59, sun_sales2#60, mon_sales2#61, tue_sales2#62, wed_sales2#63, thu_sales2#64, fri_sales2#65, sat_sales2#66] +Arguments: [d_week_seq2#58, s_store_id2#59, sun_sales2#60, mon_sales2#61, tue_sales2#62, wed_sales2#63, thu_sales2#64, fri_sales2#65, sat_sales2#66] -(40) BroadcastExchange -Input [9]: [d_week_seq2#65, s_store_id2#66, sun_sales2#67, mon_sales2#68, tue_sales2#69, wed_sales2#70, thu_sales2#71, fri_sales2#72, sat_sales2#73] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [plan_id=6] +(36) CometBroadcastHashJoin +Left output [10]: [s_store_name1#26, d_week_seq1#27, s_store_id1#28, sun_sales1#29, mon_sales1#30, tue_sales1#31, wed_sales1#32, thu_sales1#33, fri_sales1#34, sat_sales1#35] +Right output [9]: [d_week_seq2#58, s_store_id2#59, sun_sales2#60, mon_sales2#61, tue_sales2#62, wed_sales2#63, thu_sales2#64, fri_sales2#65, sat_sales2#66] +Arguments: [s_store_id1#28, d_week_seq1#27], [s_store_id2#59, (d_week_seq2#58 - 52)], Inner, BuildRight -(41) BroadcastHashJoin [codegen id : 8] -Left keys [2]: [s_store_id1#35, d_week_seq1#34] -Right keys [2]: [s_store_id2#66, (d_week_seq2#65 - 52)] -Join type: Inner -Join condition: None +(37) CometProject +Input [19]: [s_store_name1#26, d_week_seq1#27, s_store_id1#28, sun_sales1#29, mon_sales1#30, tue_sales1#31, wed_sales1#32, thu_sales1#33, fri_sales1#34, sat_sales1#35, d_week_seq2#58, s_store_id2#59, sun_sales2#60, mon_sales2#61, tue_sales2#62, wed_sales2#63, thu_sales2#64, fri_sales2#65, sat_sales2#66] +Arguments: [s_store_name1#26, s_store_id1#28, d_week_seq1#27, (sun_sales1 / sun_sales2)#67, (mon_sales1 / mon_sales2)#68, (tue_sales1 / tue_sales2)#69, (wed_sales1 / wed_sales2)#70, (thu_sales1 / thu_sales2)#71, (fri_sales1 / fri_sales2)#72, (sat_sales1 / sat_sales2)#73], [s_store_name1#26, s_store_id1#28, d_week_seq1#27, (sun_sales1#29 / sun_sales2#60) AS (sun_sales1 / sun_sales2)#67, (mon_sales1#30 / mon_sales2#61) AS (mon_sales1 / mon_sales2)#68, (tue_sales1#31 / tue_sales2#62) AS (tue_sales1 / tue_sales2)#69, (wed_sales1#32 / wed_sales2#63) AS (wed_sales1 / wed_sales2)#70, (thu_sales1#33 / thu_sales2#64) AS (thu_sales1 / thu_sales2)#71, (fri_sales1#34 / fri_sales2#65) AS (fri_sales1 / fri_sales2)#72, (sat_sales1#35 / sat_sales2#66) AS (sat_sales1 / sat_sales2)#73] -(42) Project [codegen id : 8] -Output [10]: [s_store_name1#33, s_store_id1#35, d_week_seq1#34, (sun_sales1#36 / sun_sales2#67) AS (sun_sales1 / sun_sales2)#74, (mon_sales1#37 / mon_sales2#68) AS (mon_sales1 / mon_sales2)#75, (tue_sales1#38 / tue_sales2#69) AS (tue_sales1 / tue_sales2)#76, (wed_sales1#39 / wed_sales2#70) AS (wed_sales1 / wed_sales2)#77, (thu_sales1#40 / thu_sales2#71) AS (thu_sales1 / thu_sales2)#78, (fri_sales1#41 / fri_sales2#72) AS (fri_sales1 / fri_sales2)#79, (sat_sales1#42 / sat_sales2#73) AS (sat_sales1 / sat_sales2)#80] -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#65, s_store_id2#66, sun_sales2#67, mon_sales2#68, tue_sales2#69, wed_sales2#70, thu_sales2#71, fri_sales2#72, sat_sales2#73] +(38) CometTakeOrderedAndProject +Input [10]: [s_store_name1#26, s_store_id1#28, d_week_seq1#27, (sun_sales1 / sun_sales2)#67, (mon_sales1 / mon_sales2)#68, (tue_sales1 / tue_sales2)#69, (wed_sales1 / wed_sales2)#70, (thu_sales1 / thu_sales2)#71, (fri_sales1 / fri_sales2)#72, (sat_sales1 / sat_sales2)#73] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name1#26 ASC NULLS FIRST,s_store_id1#28 ASC NULLS FIRST,d_week_seq1#27 ASC NULLS FIRST], output=[s_store_name1#26,s_store_id1#28,d_week_seq1#27,(sun_sales1 / sun_sales2)#67,(mon_sales1 / mon_sales2)#68,(tue_sales1 / tue_sales2)#69,(wed_sales1 / wed_sales2)#70,(thu_sales1 / thu_sales2)#71,(fri_sales1 / fri_sales2)#72,(sat_sales1 / sat_sales2)#73]), [s_store_name1#26, s_store_id1#28, d_week_seq1#27, (sun_sales1 / sun_sales2)#67, (mon_sales1 / mon_sales2)#68, (tue_sales1 / tue_sales2)#69, (wed_sales1 / wed_sales2)#70, (thu_sales1 / thu_sales2)#71, (fri_sales1 / fri_sales2)#72, (sat_sales1 / sat_sales2)#73], 100, [s_store_name1#26 ASC NULLS FIRST, s_store_id1#28 ASC NULLS FIRST, d_week_seq1#27 ASC NULLS FIRST], [s_store_name1#26, s_store_id1#28, d_week_seq1#27, (sun_sales1 / sun_sales2)#67, (mon_sales1 / mon_sales2)#68, (tue_sales1 / tue_sales2)#69, (wed_sales1 / wed_sales2)#70, (thu_sales1 / thu_sales2)#71, (fri_sales1 / fri_sales2)#72, (sat_sales1 / sat_sales2)#73] -(43) TakeOrderedAndProject -Input [10]: [s_store_name1#33, s_store_id1#35, d_week_seq1#34, (sun_sales1 / sun_sales2)#74, (mon_sales1 / mon_sales2)#75, (tue_sales1 / tue_sales2)#76, (wed_sales1 / wed_sales2)#77, (thu_sales1 / thu_sales2)#78, (fri_sales1 / fri_sales2)#79, (sat_sales1 / sat_sales2)#80] -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)#74, (mon_sales1 / mon_sales2)#75, (tue_sales1 / tue_sales2)#76, (wed_sales1 / wed_sales2)#77, (thu_sales1 / thu_sales2)#78, (fri_sales1 / fri_sales2)#79, (sat_sales1 / sat_sales2)#80] +(39) ColumnarToRow [codegen id : 1] +Input [10]: [s_store_name1#26, s_store_id1#28, d_week_seq1#27, (sun_sales1 / sun_sales2)#67, (mon_sales1 / mon_sales2)#68, (tue_sales1 / tue_sales2)#69, (wed_sales1 / wed_sales2)#70, (thu_sales1 / thu_sales2)#71, (fri_sales1 / fri_sales2)#72, (sat_sales1 / sat_sales2)#73] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/simplified.txt index 6c1e6cd92..bb7a85817 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/simplified.txt @@ -1,62 +1,41 @@ -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 (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] - 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,s_store_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum,d_day_name,ss_sales_price] - CometProject [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] - CometBroadcastHashJoin [ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_week_seq,d_day_name] - CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #2 - CometFilter [d_date_sk,d_week_seq,d_day_name] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_id,s_store_name] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_month_seq,d_week_seq] - CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] - InputAdapter - BroadcastExchange #5 - 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] - BroadcastHashJoin [ss_store_sk,s_store_sk] - 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 - ReusedExchange [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] #1 - InputAdapter - BroadcastExchange #6 - 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 (6) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_month_seq,d_week_seq] - CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [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)] + CometProject [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] [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)] + CometBroadcastHashJoin [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] + CometProject [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] + CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name,d_week_seq] + CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] + CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id,s_store_name] + CometHashAggregate [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,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))] + CometColumnarExchange [d_week_seq,ss_store_sk] #1 + CometHashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum,d_day_name,ss_sales_price] + CometProject [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] + CometBroadcastHashJoin [ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_week_seq,d_day_name] + CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #2 + CometFilter [d_date_sk,d_week_seq,d_day_name] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 + CometFilter [s_store_sk,s_store_id,s_store_name] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] + CometBroadcastExchange [d_week_seq] #4 + CometProject [d_week_seq] + CometFilter [d_month_seq,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] + CometBroadcastExchange [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 + CometProject [d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] + CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,d_week_seq] + CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] + CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id] + CometHashAggregate [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,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))] + ReusedExchange [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] #1 + CometBroadcastExchange [s_store_sk,s_store_id] #6 + CometFilter [s_store_sk,s_store_id] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange [d_week_seq] #7 + CometProject [d_week_seq] + CometFilter [d_month_seq,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/explain.txt index 95a19de48..0341c20d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/explain.txt @@ -1,44 +1,42 @@ == Physical Plan == -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) +* ColumnarToRow (38) ++- CometTakeOrderedAndProject (37) + +- CometFilter (36) + +- CometHashAggregate (35) + +- CometColumnarExchange (34) + +- CometHashAggregate (33) + +- CometProject (32) + +- CometBroadcastHashJoin (31) + :- 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) + +- CometBroadcastExchange (30) + +- CometProject (29) + +- CometBroadcastHashJoin (28) + :- CometFilter (20) + : +- CometScan parquet spark_catalog.default.item (19) + +- CometBroadcastExchange (27) + +- CometFilter (26) + +- CometHashAggregate (25) + +- CometColumnarExchange (24) + +- CometHashAggregate (23) + +- CometFilter (22) + +- CometScan parquet spark_catalog.default.item (21) (1) Scan parquet spark_catalog.default.customer_address @@ -129,186 +127,170 @@ Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight 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] - -(20) Scan parquet spark_catalog.default.item +(19) 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 -(21) CometFilter +(20) 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)) -(22) ColumnarToRow [codegen id : 3] -Input [3]: [i_item_sk#13, i_current_price#14, i_category#15] - -(23) Scan parquet spark_catalog.default.item +(21) 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 -(24) CometFilter +(22) CometFilter Input [2]: [i_current_price#16, i_category#17] Condition : isnotnull(i_category#17) -(25) CometHashAggregate +(23) CometHashAggregate Input [2]: [i_current_price#16, i_category#17] Keys [1]: [i_category#17] Functions [1]: [partial_avg(UnscaledValue(i_current_price#16))] -(26) ColumnarToRow [codegen id : 1] +(24) CometColumnarExchange Input [3]: [i_category#17, sum#18, count#19] +Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(27) Exchange -Input [3]: [i_category#17, sum#18, count#19] -Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, [plan_id=1] - -(28) HashAggregate [codegen id : 2] +(25) CometHashAggregate 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] -(29) Filter [codegen id : 2] -Input [2]: [avg(i_current_price)#21, i_category#17] -Condition : isnotnull(avg(i_current_price)#21) +(26) CometFilter +Input [2]: [avg(i_current_price)#20, i_category#17] +Condition : isnotnull(avg(i_current_price)#20) -(30) BroadcastExchange -Input [2]: [avg(i_current_price)#21, i_category#17] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=2] +(27) CometBroadcastExchange +Input [2]: [avg(i_current_price)#20, i_category#17] +Arguments: [avg(i_current_price)#20, i_category#17] -(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#14 as decimal(14,7)) > (1.2 * avg(i_current_price)#21)) +(28) CometBroadcastHashJoin +Left output [3]: [i_item_sk#13, i_current_price#14, i_category#15] +Right output [2]: [avg(i_current_price)#20, i_category#17] +Arguments: [i_category#15], [i_category#17], Inner, (cast(i_current_price#14 as decimal(14,7)) > (1.2 * avg(i_current_price)#20)), BuildRight -(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] +(29) CometProject +Input [5]: [i_item_sk#13, i_current_price#14, i_category#15, avg(i_current_price)#20, i_category#17] +Arguments: [i_item_sk#13], [i_item_sk#13] -(33) BroadcastExchange +(30) CometBroadcastExchange Input [1]: [i_item_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: [i_item_sk#13] -(34) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#5] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None +(31) CometBroadcastHashJoin +Left output [2]: [ca_state#2, ss_item_sk#5] +Right output [1]: [i_item_sk#13] +Arguments: [ss_item_sk#5], [i_item_sk#13], Inner, BuildRight -(35) Project [codegen id : 4] -Output [1]: [ca_state#2] +(32) CometProject Input [3]: [ca_state#2, ss_item_sk#5, i_item_sk#13] +Arguments: [ca_state#2], [ca_state#2] -(36) HashAggregate [codegen id : 4] +(33) CometHashAggregate Input [1]: [ca_state#2] Keys [1]: [ca_state#2] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#22] -Results [2]: [ca_state#2, count#23] -(37) Exchange -Input [2]: [ca_state#2, count#23] -Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(34) CometColumnarExchange +Input [2]: [ca_state#2, count#21] +Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(38) HashAggregate [codegen id : 5] -Input [2]: [ca_state#2, count#23] +(35) CometHashAggregate +Input [2]: [ca_state#2, count#21] Keys [1]: [ca_state#2] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#24] -Results [2]: [ca_state#2 AS state#25, count(1)#24 AS cnt#26] -(39) Filter [codegen id : 5] -Input [2]: [state#25, cnt#26] -Condition : (cnt#26 >= 10) +(36) CometFilter +Input [2]: [state#22, cnt#23] +Condition : (cnt#23 >= 10) -(40) TakeOrderedAndProject -Input [2]: [state#25, cnt#26] -Arguments: 100, [cnt#26 ASC NULLS FIRST], [state#25, cnt#26] +(37) CometTakeOrderedAndProject +Input [2]: [state#22, cnt#23] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[cnt#23 ASC NULLS FIRST], output=[state#22,cnt#23]), [state#22, cnt#23], 100, [cnt#23 ASC NULLS FIRST], [state#22, cnt#23] + +(38) ColumnarToRow [codegen id : 1] +Input [2]: [state#22, cnt#23] ===== Subqueries ===== 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) +BroadcastExchange (43) ++- * ColumnarToRow (42) + +- CometProject (41) + +- CometFilter (40) + +- CometScan parquet spark_catalog.default.date_dim (39) -(41) Scan parquet spark_catalog.default.date_dim +(39) 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 -(42) CometFilter +(40) 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)) -(43) CometProject +(41) CometProject Input [2]: [d_date_sk#9, d_month_seq#10] Arguments: [d_date_sk#9], [d_date_sk#9] -(44) ColumnarToRow [codegen id : 1] +(42) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(45) BroadcastExchange +(43) BroadcastExchange Input [1]: [d_date_sk#9] -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 = 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) +Subquery:2 Hosting operator id = 40 Hosting Expression = Subquery scalar-subquery#11, [id=#12] +* ColumnarToRow (50) ++- CometHashAggregate (49) + +- CometColumnarExchange (48) + +- CometHashAggregate (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan parquet spark_catalog.default.date_dim (44) -(46) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#27, d_year#28, d_moy#29] +(44) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_month_seq#24, d_year#25, d_moy#26] 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 -(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)) +(45) CometFilter +Input [3]: [d_month_seq#24, d_year#25, d_moy#26] +Condition : (((isnotnull(d_year#25) AND isnotnull(d_moy#26)) AND (d_year#25 = 2000)) AND (d_moy#26 = 1)) -(48) CometProject -Input [3]: [d_month_seq#27, d_year#28, d_moy#29] -Arguments: [d_month_seq#27], [d_month_seq#27] +(46) CometProject +Input [3]: [d_month_seq#24, d_year#25, d_moy#26] +Arguments: [d_month_seq#24], [d_month_seq#24] -(49) CometHashAggregate -Input [1]: [d_month_seq#27] -Keys [1]: [d_month_seq#27] +(47) CometHashAggregate +Input [1]: [d_month_seq#24] +Keys [1]: [d_month_seq#24] Functions: [] -(50) ColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#27] - -(51) Exchange -Input [1]: [d_month_seq#27] -Arguments: hashpartitioning(d_month_seq#27, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(48) CometColumnarExchange +Input [1]: [d_month_seq#24] +Arguments: hashpartitioning(d_month_seq#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(52) HashAggregate [codegen id : 2] -Input [1]: [d_month_seq#27] -Keys [1]: [d_month_seq#27] +(49) CometHashAggregate +Input [1]: [d_month_seq#24] +Keys [1]: [d_month_seq#24] Functions: [] -Aggregate Attributes: [] -Results [1]: [d_month_seq#27] + +(50) ColumnarToRow [codegen id : 1] +Input [1]: [d_month_seq#24] 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-spark3_5/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/simplified.txt index 824670101..0e983b96a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/simplified.txt @@ -1,73 +1,59 @@ -TakeOrderedAndProject [cnt,state] - WholeStageCodegen (5) - Filter [cnt] - HashAggregate [ca_state,count] [count(1),state,cnt,count] - InputAdapter - Exchange [ca_state] #1 - WholeStageCodegen (4) - HashAggregate [ca_state] [count,count] - Project [ca_state] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometProject [ca_state,ss_item_sk] - CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] - CometProject [ca_state,ss_item_sk,ss_sold_date_sk] - CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] - CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_customer_sk,ss_sold_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 #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [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_month_seq,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 [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [d_month_seq] #2 - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - Project [i_item_sk] - BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_current_price,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - InputAdapter - 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] #9 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_category,sum,count,i_current_price] - CometFilter [i_current_price,i_category] - CometScan parquet spark_catalog.default.item [i_current_price,i_category] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [state,cnt] + CometFilter [state,cnt] + CometHashAggregate [state,cnt,ca_state,count,count(1)] + CometColumnarExchange [ca_state] #1 + CometHashAggregate [ca_state,count] + CometProject [ca_state] + CometBroadcastHashJoin [ca_state,ss_item_sk,i_item_sk] + CometProject [ca_state,ss_item_sk] + CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] + CometProject [ca_state,ss_item_sk,ss_sold_date_sk] + CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] + CometProject [ca_state,c_customer_sk] + CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_customer_sk,ss_sold_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 #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + Subquery #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [d_month_seq] + CometColumnarExchange [d_month_seq] #5 + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_month_seq,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 [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + ReusedSubquery [d_month_seq] #2 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk] #7 + CometProject [i_item_sk] + CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] + CometFilter [i_item_sk,i_current_price,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] + CometBroadcastExchange [avg(i_current_price),i_category] #8 + CometFilter [avg(i_current_price),i_category] + CometHashAggregate [avg(i_current_price),i_category,sum,count,avg(UnscaledValue(i_current_price))] + CometColumnarExchange [i_category] #9 + CometHashAggregate [i_category,sum,count,i_current_price] + CometFilter [i_current_price,i_category] + CometScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/explain.txt index 0f61456dd..0aeb4e2b4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/explain.txt @@ -1,67 +1,65 @@ == Physical Plan == -TakeOrderedAndProject (63) -+- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- Union (59) - :- * HashAggregate (28) - : +- Exchange (27) - : +- * 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) - : +- * 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) - +- * 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) +* ColumnarToRow (61) ++- CometTakeOrderedAndProject (60) + +- CometHashAggregate (59) + +- CometColumnarExchange (58) + +- CometHashAggregate (57) + +- CometUnion (56) + :- CometHashAggregate (27) + : +- CometColumnarExchange (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) + :- CometHashAggregate (41) + : +- CometColumnarExchange (40) + : +- CometHashAggregate (39) + : +- CometProject (38) + : +- CometBroadcastHashJoin (37) + : :- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometProject (32) + : : : +- CometBroadcastHashJoin (31) + : : : :- CometFilter (29) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (28) + : : : +- ReusedExchange (30) + : : +- ReusedExchange (33) + : +- ReusedExchange (36) + +- CometHashAggregate (55) + +- CometColumnarExchange (54) + +- CometHashAggregate (53) + +- CometProject (52) + +- CometBroadcastHashJoin (51) + :- CometProject (49) + : +- CometBroadcastHashJoin (48) + : :- CometProject (46) + : : +- CometBroadcastHashJoin (45) + : : :- CometFilter (43) + : : : +- CometScan parquet spark_catalog.default.web_sales (42) + : : +- ReusedExchange (44) + : +- ReusedExchange (47) + +- ReusedExchange (50) (1) Scan parquet spark_catalog.default.store_sales @@ -185,212 +183,199 @@ 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))] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarExchange Input [2]: [i_item_id#12, sum#15] +Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(27) Exchange -Input [2]: [i_item_id#12, sum#15] -Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=1] - -(28) HashAggregate [codegen id : 2] +(27) CometHashAggregate 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))#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#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] +(28) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_bill_addr_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#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#22)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#20)] PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(30) CometFilter -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)) +(29) CometFilter +Input [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] +Condition : (isnotnull(cs_bill_addr_sk#16) AND isnotnull(cs_item_sk#17)) -(31) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#23] +(30) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#21] -(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 +(31) CometBroadcastHashJoin +Left output [4]: [cs_bill_addr_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 -(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] +(32) CometProject +Input [5]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19, d_date_sk#21] +Arguments: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18], [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18] -(34) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#24] +(33) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#22] -(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 +(34) CometBroadcastHashJoin +Left output [3]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18] +Right output [1]: [ca_address_sk#22] +Arguments: [cs_bill_addr_sk#16], [ca_address_sk#22], 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] +(35) CometProject +Input [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, ca_address_sk#22] +Arguments: [cs_item_sk#17, cs_ext_sales_price#18], [cs_item_sk#17, cs_ext_sales_price#18] -(37) ReusedExchange [Reuses operator id: 22] -Output [2]: [i_item_sk#25, i_item_id#26] +(36) ReusedExchange [Reuses operator id: 22] +Output [2]: [i_item_sk#23, i_item_id#24] -(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 +(37) CometBroadcastHashJoin +Left output [2]: [cs_item_sk#17, cs_ext_sales_price#18] +Right output [2]: [i_item_sk#23, i_item_id#24] +Arguments: [cs_item_sk#17], [i_item_sk#23], 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] +(38) CometProject +Input [4]: [cs_item_sk#17, cs_ext_sales_price#18, i_item_sk#23, i_item_id#24] +Arguments: [cs_ext_sales_price#18, i_item_id#24], [cs_ext_sales_price#18, i_item_id#24] -(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))] +(39) CometHashAggregate +Input [2]: [cs_ext_sales_price#18, i_item_id#24] +Keys [1]: [i_item_id#24] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#18))] -(41) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_id#26, sum#27] +(40) CometColumnarExchange +Input [2]: [i_item_id#24, sum#25] +Arguments: hashpartitioning(i_item_id#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(42) Exchange -Input [2]: [i_item_id#26, sum#27] -Arguments: hashpartitioning(i_item_id#26, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(41) CometHashAggregate +Input [2]: [i_item_id#24, sum#25] +Keys [1]: [i_item_id#24] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#18))] -(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] +(42) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] 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#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#30)] 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)) +(43) CometFilter +Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] +Condition : (isnotnull(ws_bill_addr_sk#27) AND isnotnull(ws_item_sk#26)) -(46) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#35] +(44) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#31] -(47) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] -Right output [1]: [d_date_sk#35] -Arguments: [ws_sold_date_sk#33], [d_date_sk#35], Inner, BuildRight +(45) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] +Right output [1]: [d_date_sk#31] +Arguments: [ws_sold_date_sk#29], [d_date_sk#31], Inner, BuildRight -(48) CometProject -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] -Arguments: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32], [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] +(46) CometProject +Input [5]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29, d_date_sk#31] +Arguments: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28], [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28] -(49) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#36] +(47) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#32] -(50) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] -Right output [1]: [ca_address_sk#36] -Arguments: [ws_bill_addr_sk#31], [ca_address_sk#36], Inner, BuildRight +(48) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28] +Right output [1]: [ca_address_sk#32] +Arguments: [ws_bill_addr_sk#27], [ca_address_sk#32], Inner, BuildRight -(51) CometProject -Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ca_address_sk#36] -Arguments: [ws_item_sk#30, ws_ext_sales_price#32], [ws_item_sk#30, ws_ext_sales_price#32] +(49) CometProject +Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ca_address_sk#32] +Arguments: [ws_item_sk#26, ws_ext_sales_price#28], [ws_item_sk#26, ws_ext_sales_price#28] -(52) ReusedExchange [Reuses operator id: 22] -Output [2]: [i_item_sk#37, i_item_id#38] +(50) ReusedExchange [Reuses operator id: 22] +Output [2]: [i_item_sk#33, i_item_id#34] -(53) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_ext_sales_price#32] -Right output [2]: [i_item_sk#37, i_item_id#38] -Arguments: [ws_item_sk#30], [i_item_sk#37], Inner, BuildRight +(51) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#26, ws_ext_sales_price#28] +Right output [2]: [i_item_sk#33, i_item_id#34] +Arguments: [ws_item_sk#26], [i_item_sk#33], Inner, BuildRight -(54) CometProject -Input [4]: [ws_item_sk#30, ws_ext_sales_price#32, i_item_sk#37, i_item_id#38] -Arguments: [ws_ext_sales_price#32, i_item_id#38], [ws_ext_sales_price#32, i_item_id#38] +(52) CometProject +Input [4]: [ws_item_sk#26, ws_ext_sales_price#28, i_item_sk#33, i_item_id#34] +Arguments: [ws_ext_sales_price#28, i_item_id#34], [ws_ext_sales_price#28, i_item_id#34] -(55) CometHashAggregate -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))] +(53) CometHashAggregate +Input [2]: [ws_ext_sales_price#28, i_item_id#34] +Keys [1]: [i_item_id#34] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#28))] -(56) ColumnarToRow [codegen id : 5] -Input [2]: [i_item_id#38, sum#39] +(54) CometColumnarExchange +Input [2]: [i_item_id#34, sum#35] +Arguments: hashpartitioning(i_item_id#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(57) Exchange -Input [2]: [i_item_id#38, sum#39] -Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(58) HashAggregate [codegen id : 6] -Input [2]: [i_item_id#38, sum#39] -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))#40] -Results [2]: [i_item_id#38, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#32))#40,17,2) AS total_sales#41] +(55) CometHashAggregate +Input [2]: [i_item_id#34, sum#35] +Keys [1]: [i_item_id#34] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#28))] -(59) Union +(56) CometUnion +Child 0 Input [2]: [i_item_id#12, total_sales#36] +Child 1 Input [2]: [i_item_id#24, total_sales#37] +Child 2 Input [2]: [i_item_id#34, total_sales#38] -(60) HashAggregate [codegen id : 7] -Input [2]: [i_item_id#12, total_sales#17] +(57) CometHashAggregate +Input [2]: [i_item_id#12, total_sales#36] Keys [1]: [i_item_id#12] -Functions [1]: [partial_sum(total_sales#17)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [3]: [i_item_id#12, sum#44, isEmpty#45] +Functions [1]: [partial_sum(total_sales#36)] -(61) Exchange -Input [3]: [i_item_id#12, sum#44, isEmpty#45] -Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(58) CometColumnarExchange +Input [3]: [i_item_id#12, sum#39, isEmpty#40] +Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(62) HashAggregate [codegen id : 8] -Input [3]: [i_item_id#12, sum#44, isEmpty#45] +(59) CometHashAggregate +Input [3]: [i_item_id#12, sum#39, isEmpty#40] Keys [1]: [i_item_id#12] -Functions [1]: [sum(total_sales#17)] -Aggregate Attributes [1]: [sum(total_sales#17)#46] -Results [2]: [i_item_id#12, sum(total_sales#17)#46 AS total_sales#47] +Functions [1]: [sum(total_sales#36)] + +(60) CometTakeOrderedAndProject +Input [2]: [i_item_id#12, total_sales#41] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#12 ASC NULLS FIRST,total_sales#41 ASC NULLS FIRST], output=[i_item_id#12,total_sales#41]), [i_item_id#12, total_sales#41], 100, [i_item_id#12 ASC NULLS FIRST, total_sales#41 ASC NULLS FIRST], [i_item_id#12, total_sales#41] -(63) TakeOrderedAndProject -Input [2]: [i_item_id#12, total_sales#47] -Arguments: 100, [i_item_id#12 ASC NULLS FIRST, total_sales#47 ASC NULLS FIRST], [i_item_id#12, total_sales#47] +(61) ColumnarToRow [codegen id : 1] +Input [2]: [i_item_id#12, total_sales#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (68) -+- * ColumnarToRow (67) - +- CometProject (66) - +- CometFilter (65) - +- CometScan parquet spark_catalog.default.date_dim (64) +BroadcastExchange (66) ++- * ColumnarToRow (65) + +- CometProject (64) + +- CometFilter (63) + +- CometScan parquet spark_catalog.default.date_dim (62) -(64) Scan parquet spark_catalog.default.date_dim +(62) Scan parquet spark_catalog.default.date_dim 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 +(63) 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)) -(66) CometProject +(64) CometProject 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] +(65) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(68) BroadcastExchange +(66) BroadcastExchange Input [1]: [d_date_sk#6] 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#21 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 28 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/simplified.txt index 7cfcb75da..871f477b7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/simplified.txt @@ -1,89 +1,73 @@ -TakeOrderedAndProject [i_item_id,total_sales] - WholeStageCodegen (8) - HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - InputAdapter - Exchange [i_item_id] #1 - WholeStageCodegen (7) - HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (2) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_item_id] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,sum,ss_ext_sales_price] - CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,total_sales] + CometHashAggregate [i_item_id,total_sales,sum,isEmpty,sum(total_sales)] + CometColumnarExchange [i_item_id] #1 + CometHashAggregate [i_item_id,sum,isEmpty,total_sales] + CometUnion [i_item_id,total_sales] + CometHashAggregate [i_item_id,total_sales,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange [i_item_id] #2 + CometHashAggregate [i_item_id,sum,ss_ext_sales_price] + CometProject [ss_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ca_address_sk] #5 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [i_item_id] #7 - CometProject [i_item_id] - CometFilter [i_item_id,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] #8 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,sum,cs_ext_sales_price] - CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] - CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] - CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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 - 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] #9 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,sum,ws_ext_sales_price] - CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] - CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] - CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ca_address_sk] #5 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange [i_item_id] #7 + CometProject [i_item_id] + CometFilter [i_item_id,i_category] + CometScan parquet spark_catalog.default.item [i_item_id,i_category] + CometHashAggregate [i_item_id,total_sales,sum,sum(UnscaledValue(cs_ext_sales_price))] + CometColumnarExchange [i_item_id] #8 + CometHashAggregate [i_item_id,sum,cs_ext_sales_price] + CometProject [cs_ext_sales_price,i_item_id] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] + CometProject [cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] + CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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 + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #6 + CometHashAggregate [i_item_id,total_sales,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometColumnarExchange [i_item_id] #9 + CometHashAggregate [i_item_id,sum,ws_ext_sales_price] + CometProject [ws_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] + CometProject [ws_item_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] + CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 + 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-spark3_5/q61/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/explain.txt index 1567198fe..3bbacfbb9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == * Project (65) +- * BroadcastNestedLoopJoin Inner BuildRight (64) - :- * HashAggregate (41) - : +- Exchange (40) - : +- * ColumnarToRow (39) + :- * ColumnarToRow (41) + : +- CometHashAggregate (40) + : +- CometColumnarExchange (39) : +- CometHashAggregate (38) : +- CometProject (37) : +- CometBroadcastHashJoin (36) @@ -43,9 +43,9 @@ : +- CometFilter (33) : +- CometScan parquet spark_catalog.default.item (32) +- BroadcastExchange (63) - +- * HashAggregate (62) - +- Exchange (61) - +- * ColumnarToRow (60) + +- * ColumnarToRow (62) + +- CometHashAggregate (61) + +- CometColumnarExchange (60) +- CometHashAggregate (59) +- CometProject (58) +- CometBroadcastHashJoin (57) @@ -247,122 +247,118 @@ Input [1]: [ss_ext_sales_price#5] Keys: [] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] -(39) ColumnarToRow [codegen id : 1] +(39) CometColumnarExchange Input [1]: [sum#23] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(40) Exchange -Input [1]: [sum#23] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] - -(41) HashAggregate [codegen id : 4] +(40) CometHashAggregate Input [1]: [sum#23] Keys: [] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -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] + +(41) ColumnarToRow [codegen id : 2] +Input [1]: [promotions#24] (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] +Output [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#30), dynamicpruningexpression(ss_sold_date_sk#30 IN dynamicpruning#31)] +PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#30)] PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (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)) +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)) (44) ReusedExchange [Reuses operator id: 6] -Output [1]: [s_store_sk#32] +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 +Left output [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] +Right output [1]: [s_store_sk#31] +Arguments: [ss_store_sk#27], [s_store_sk#31], Inner, BuildRight (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] +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] +Arguments: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29], [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29] (47) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#33] +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 +Left output [4]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29] +Right output [1]: [d_date_sk#32] +Arguments: [ss_sold_date_sk#29], [d_date_sk#32], Inner, BuildRight (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] +Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#32] +Arguments: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28], [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28] (50) ReusedExchange [Reuses operator id: 23] -Output [2]: [c_customer_sk#34, c_current_addr_sk#35] +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 +Left output [3]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28] +Right output [2]: [c_customer_sk#33, c_current_addr_sk#34] +Arguments: [ss_customer_sk#26], [c_customer_sk#33], Inner, BuildRight (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] +Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, c_customer_sk#33, c_current_addr_sk#34] +Arguments: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34], [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34] (53) ReusedExchange [Reuses operator id: 29] -Output [1]: [ca_address_sk#36] +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 +Left output [3]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34] +Right output [1]: [ca_address_sk#35] +Arguments: [c_current_addr_sk#34], [ca_address_sk#35], Inner, BuildRight (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] +Input [4]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34, ca_address_sk#35] +Arguments: [ss_item_sk#25, ss_ext_sales_price#28], [ss_item_sk#25, ss_ext_sales_price#28] (56) ReusedExchange [Reuses operator id: 35] -Output [1]: [i_item_sk#37] +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 +Left output [2]: [ss_item_sk#25, ss_ext_sales_price#28] +Right output [1]: [i_item_sk#36] +Arguments: [ss_item_sk#25], [i_item_sk#36], Inner, BuildRight (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] +Input [3]: [ss_item_sk#25, ss_ext_sales_price#28, i_item_sk#36] +Arguments: [ss_ext_sales_price#28], [ss_ext_sales_price#28] (59) CometHashAggregate -Input [1]: [ss_ext_sales_price#29] +Input [1]: [ss_ext_sales_price#28] Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#29))] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#28))] -(60) ColumnarToRow [codegen id : 2] -Input [1]: [sum#38] +(60) CometColumnarExchange +Input [1]: [sum#37] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(61) Exchange -Input [1]: [sum#38] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] - -(62) HashAggregate [codegen id : 3] -Input [1]: [sum#38] +(61) CometHashAggregate +Input [1]: [sum#37] Keys: [] -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] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#28))] + +(62) ColumnarToRow [codegen id : 1] +Input [1]: [total#38] (63) BroadcastExchange -Input [1]: [total#40] +Input [1]: [total#38] Arguments: IdentityBroadcastMode, [plan_id=3] -(64) BroadcastNestedLoopJoin [codegen id : 4] +(64) BroadcastNestedLoopJoin [codegen id : 2] Join type: Inner Join condition: None -(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] +(65) Project [codegen id : 2] +Output [3]: [promotions#24, total#38, ((cast(promotions#24 as decimal(15,4)) / cast(total#38 as decimal(15,4))) * 100) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#39] +Input [2]: [promotions#24, total#38] ===== Subqueries ===== @@ -396,6 +392,6 @@ Input [1]: [d_date_sk#14] Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -Subquery:2 Hosting operator id = 42 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 42 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/simplified.txt index 1b2af33b0..b95b0d616 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/simplified.txt @@ -1,83 +1,79 @@ -WholeStageCodegen (4) +WholeStageCodegen (2) Project [promotions,total] BroadcastNestedLoopJoin - HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),promotions,sum] + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [sum,ss_ext_sales_price] - CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] - CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk] - CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk] - CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,p_promo_sk] - CometProject [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk] #3 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_gmt_offset] - CometScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] - CometBroadcastExchange [p_promo_sk] #4 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #6 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk] #7 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk] #8 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_category] + CometHashAggregate [promotions,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange #1 + CometHashAggregate [sum,ss_ext_sales_price] + CometProject [ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk] + CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,p_promo_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk] #3 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_gmt_offset] + CometScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] + CometBroadcastExchange [p_promo_sk] #4 + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] + CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #6 + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange [ca_address_sk] #7 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometBroadcastExchange [i_item_sk] #8 + CometProject [i_item_sk] + CometFilter [i_item_sk,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_category] InputAdapter BroadcastExchange #9 - WholeStageCodegen (3) - HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),total,sum] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange #10 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometHashAggregate [sum,ss_ext_sales_price] - CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] - CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk] - CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk] - CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_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 - 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 + CometHashAggregate [total,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange #10 + CometHashAggregate [sum,ss_ext_sales_price] + CometProject [ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk] + CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_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 + 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-spark3_5/q62/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/explain.txt index 21d44db2c..f58411101 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (28) -+- * HashAggregate (27) - +- Exchange (26) - +- * ColumnarToRow (25) +* ColumnarToRow (28) ++- CometTakeOrderedAndProject (27) + +- CometHashAggregate (26) + +- CometColumnarExchange (25) +- CometHashAggregate (24) +- CometProject (23) +- CometBroadcastHashJoin (22) @@ -145,21 +145,19 @@ Input [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#9, web_name#11, _group 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)] -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(27) HashAggregate [codegen id : 2] +(26) CometHashAggregate 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)#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] -(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] +(27) CometTakeOrderedAndProject +Input [8]: [substr(w_warehouse_name, 1, 20)#20, sm_type#9, web_name#11, 30 days #21, 31 - 60 days #22, 61 - 90 days #23, 91 - 120 days #24, >120 days #25] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#20 ASC NULLS FIRST,sm_type#9 ASC NULLS FIRST,web_name#11 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#20,sm_type#9,web_name#11,30 days #21,31 - 60 days #22,61 - 90 days #23,91 - 120 days #24,>120 days #25]), [substr(w_warehouse_name, 1, 20)#20, sm_type#9, web_name#11, 30 days #21, 31 - 60 days #22, 61 - 90 days #23, 91 - 120 days #24, >120 days #25], 100, [substr(w_warehouse_name, 1, 20)#20 ASC NULLS FIRST, sm_type#9 ASC NULLS FIRST, web_name#11 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#20, sm_type#9, web_name#11, 30 days #21, 31 - 60 days #22, 61 - 90 days #23, 91 - 120 days #24, >120 days #25] + +(28) ColumnarToRow [codegen id : 1] +Input [8]: [substr(w_warehouse_name, 1, 20)#20, sm_type#9, web_name#11, 30 days #21, 31 - 60 days #22, 61 - 90 days #23, 91 - 120 days #24, >120 days #25] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/simplified.txt index 0b4fc61a1..c7bcf72f0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/simplified.txt @@ -1,32 +1,30 @@ -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 (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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum,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,ws_sold_date_sk,w_warehouse_name,sm_type,web_name,d_date_sk] - CometProject [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name] - CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_site_sk,web_name] - CometProject [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type] - CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] - CometProject [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name] - CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name] - CometFilter [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_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] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 - CometFilter [sm_ship_mode_sk,sm_type] - CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange [web_site_sk,web_name] #4 - CometFilter [web_site_sk,web_name] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_name] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] + CometHashAggregate [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,_groupingexpression,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)] + CometColumnarExchange [_groupingexpression,sm_type,web_name] #1 + CometHashAggregate [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum,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,ws_sold_date_sk,w_warehouse_name,sm_type,web_name,d_date_sk] + CometProject [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_site_sk,web_name] + CometProject [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] + CometProject [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_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] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 + CometFilter [w_warehouse_sk,w_warehouse_name] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 + CometFilter [sm_ship_mode_sk,sm_type] + CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] + CometBroadcastExchange [web_site_sk,web_name] #4 + CometFilter [web_site_sk,web_name] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_name] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + 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-spark3_5/q63/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/explain.txt index 621e9c8c6..5fd263660 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/explain.txt @@ -3,11 +3,11 @@ TakeOrderedAndProject (29) +- * Project (28) +- * Filter (27) +- Window (26) - +- * Sort (25) - +- Exchange (24) - +- * HashAggregate (23) - +- Exchange (22) - +- * ColumnarToRow (21) + +- * ColumnarToRow (25) + +- CometSort (24) + +- CometColumnarExchange (23) + +- CometHashAggregate (22) + +- CometColumnarExchange (21) +- CometHashAggregate (20) +- CometProject (19) +- CometBroadcastHashJoin (18) @@ -127,43 +127,41 @@ 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))] -(21) ColumnarToRow [codegen id : 1] +(21) CometColumnarExchange Input [3]: [i_manager_id#5, d_moy#17, sum#19] +Arguments: hashpartitioning(i_manager_id#5, d_moy#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(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] - -(23) HashAggregate [codegen id : 2] +(22) CometHashAggregate 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] -(24) Exchange -Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] -Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(23) CometColumnarExchange +Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] +Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(24) CometSort +Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] +Arguments: [i_manager_id#5, sum_sales#20, _w0#21], [i_manager_id#5 ASC NULLS FIRST] -(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) ColumnarToRow [codegen id : 1] +Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] (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] +Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] +Arguments: [avg(_w0#21) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_manager_id#5] -(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) Filter [codegen id : 2] +Input [4]: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22] +Condition : CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) ELSE false END -(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) Project [codegen id : 2] +Output [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] +Input [4]: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22] (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] +Input [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] +Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST], [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/simplified.txt index b2033c7a9..1b538c9ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/simplified.txt @@ -1,45 +1,41 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] - WholeStageCodegen (4) + WholeStageCodegen (2) Project [i_manager_id,sum_sales,avg_monthly_sales] Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_manager_id] - WholeStageCodegen (3) - Sort [i_manager_id] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [i_manager_id] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_manager_id,d_moy,sum,ss_sales_price] - CometProject [i_manager_id,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] - CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] - CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_item_sk,i_manager_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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_date_sk,d_month_seq,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] + CometSort [i_manager_id,sum_sales,_w0] + CometColumnarExchange [i_manager_id] #1 + CometHashAggregate [i_manager_id,sum_sales,_w0,d_moy,sum,sum(UnscaledValue(ss_sales_price))] + CometColumnarExchange [i_manager_id,d_moy] #2 + CometHashAggregate [i_manager_id,d_moy,sum,ss_sales_price] + CometProject [i_manager_id,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] + CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] + CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_item_sk,i_manager_id] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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_date_sk,d_month_seq,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometBroadcastExchange [d_date_sk,d_moy] #5 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_month_seq,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometBroadcastExchange [s_store_sk] #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-spark3_5/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/explain.txt index bd491e60f..b4e05a864 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/explain.txt @@ -1,185 +1,180 @@ == Physical Plan == -* Sort (181) -+- Exchange (180) - +- * Project (179) - +- * SortMergeJoin Inner (178) - :- * Sort (110) - : +- Exchange (109) - : +- * HashAggregate (108) - : +- * HashAggregate (107) - : +- * Project (106) - : +- * BroadcastHashJoin Inner BuildRight (105) - : :- * Project (99) - : : +- * BroadcastHashJoin Inner BuildRight (98) - : : :- * Project (96) - : : : +- * BroadcastHashJoin Inner BuildRight (95) - : : : :- * Project (90) - : : : : +- * BroadcastHashJoin Inner BuildRight (89) - : : : : :- * Project (87) - : : : : : +- * BroadcastHashJoin Inner BuildRight (86) - : : : : : :- * Project (81) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (80) - : : : : : : :- * Project (78) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (77) - : : : : : : : :- * Project (72) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (71) - : : : : : : : : :- * Project (66) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (65) - : : : : : : : : : :- * Project (63) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (62) - : : : : : : : : : : :- * Project (57) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (56) - : : : : : : : : : : : :- * Project (54) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (53) - : : : : : : : : : : : : :- * Project (48) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (47) - : : : : : : : : : : : : : :- * Project (42) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : : : : : : : : : : : :- * Project (36) - : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (35) - : : : : : : : : : : : : : : : :- * Project (33) - : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (32) - : : : : : : : : : : : : : : : : :- * Sort (11) - : : : : : : : : : : : : : : : : : +- Exchange (10) - : : : : : : : : : : : : : : : : : +- * ColumnarToRow (9) - : : : : : : : : : : : : : : : : : +- CometProject (8) - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) - : : : : : : : : : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- CometProject (6) - : : : : : : : : : : : : : : : : : +- CometFilter (5) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : : : : : : : : : : : : : : : +- * Sort (31) - : : : : : : : : : : : : : : : : +- * Project (30) - : : : : : : : : : : : : : : : : +- * Filter (29) - : : : : : : : : : : : : : : : : +- * HashAggregate (28) - : : : : : : : : : : : : : : : : +- Exchange (27) - : : : : : : : : : : : : : : : : +- * HashAggregate (26) - : : : : : : : : : : : : : : : : +- * Project (25) - : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (24) - : : : : : : : : : : : : : : : : :- * Sort (17) - : : : : : : : : : : : : : : : : : +- Exchange (16) - : : : : : : : : : : : : : : : : : +- * ColumnarToRow (15) - : : : : : : : : : : : : : : : : : +- CometProject (14) - : : : : : : : : : : : : : : : : : +- CometFilter (13) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (12) - : : : : : : : : : : : : : : : : +- * Sort (23) - : : : : : : : : : : : : : : : : +- Exchange (22) - : : : : : : : : : : : : : : : : +- * ColumnarToRow (21) - : : : : : : : : : : : : : : : : +- CometProject (20) - : : : : : : : : : : : : : : : : +- CometFilter (19) - : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (18) - : : : : : : : : : : : : : : : +- ReusedExchange (34) - : : : : : : : : : : : : : : +- BroadcastExchange (40) - : : : : : : : : : : : : : : +- * ColumnarToRow (39) - : : : : : : : : : : : : : : +- CometFilter (38) - : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (37) - : : : : : : : : : : : : : +- BroadcastExchange (46) - : : : : : : : : : : : : : +- * ColumnarToRow (45) - : : : : : : : : : : : : : +- CometFilter (44) - : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (43) - : : : : : : : : : : : : +- BroadcastExchange (52) - : : : : : : : : : : : : +- * ColumnarToRow (51) - : : : : : : : : : : : : +- CometFilter (50) - : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (49) - : : : : : : : : : : : +- ReusedExchange (55) - : : : : : : : : : : +- BroadcastExchange (61) - : : : : : : : : : : +- * ColumnarToRow (60) - : : : : : : : : : : +- CometFilter (59) - : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (58) - : : : : : : : : : +- ReusedExchange (64) - : : : : : : : : +- BroadcastExchange (70) - : : : : : : : : +- * ColumnarToRow (69) - : : : : : : : : +- CometFilter (68) - : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (67) - : : : : : : : +- BroadcastExchange (76) - : : : : : : : +- * ColumnarToRow (75) - : : : : : : : +- CometFilter (74) - : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (73) - : : : : : : +- ReusedExchange (79) - : : : : : +- BroadcastExchange (85) - : : : : : +- * ColumnarToRow (84) - : : : : : +- CometFilter (83) - : : : : : +- CometScan parquet spark_catalog.default.customer_address (82) - : : : : +- ReusedExchange (88) - : : : +- BroadcastExchange (94) - : : : +- * ColumnarToRow (93) - : : : +- CometFilter (92) - : : : +- CometScan parquet spark_catalog.default.income_band (91) - : : +- ReusedExchange (97) - : +- BroadcastExchange (104) - : +- * ColumnarToRow (103) - : +- CometProject (102) - : +- CometFilter (101) - : +- CometScan parquet spark_catalog.default.item (100) - +- * Sort (177) - +- Exchange (176) - +- * HashAggregate (175) - +- * HashAggregate (174) - +- * Project (173) - +- * BroadcastHashJoin Inner BuildRight (172) - :- * Project (170) - : +- * BroadcastHashJoin Inner BuildRight (169) - : :- * Project (167) - : : +- * BroadcastHashJoin Inner BuildRight (166) - : : :- * Project (164) - : : : +- * BroadcastHashJoin Inner BuildRight (163) - : : : :- * Project (161) - : : : : +- * BroadcastHashJoin Inner BuildRight (160) - : : : : :- * Project (158) - : : : : : +- * BroadcastHashJoin Inner BuildRight (157) - : : : : : :- * Project (155) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (154) - : : : : : : :- * Project (152) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (151) - : : : : : : : :- * Project (149) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (148) - : : : : : : : : :- * Project (146) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (145) - : : : : : : : : : :- * Project (143) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (142) - : : : : : : : : : : :- * Project (140) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (139) - : : : : : : : : : : : :- * Project (137) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (136) - : : : : : : : : : : : : :- * Project (134) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (133) - : : : : : : : : : : : : : :- * Project (131) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (130) - : : : : : : : : : : : : : : :- * Project (128) - : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (127) - : : : : : : : : : : : : : : : :- * Sort (121) - : : : : : : : : : : : : : : : : +- Exchange (120) - : : : : : : : : : : : : : : : : +- * ColumnarToRow (119) - : : : : : : : : : : : : : : : : +- CometProject (118) - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (117) - : : : : : : : : : : : : : : : : :- CometBroadcastExchange (113) - : : : : : : : : : : : : : : : : : +- CometFilter (112) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (111) - : : : : : : : : : : : : : : : : +- CometProject (116) - : : : : : : : : : : : : : : : : +- CometFilter (115) - : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (114) - : : : : : : : : : : : : : : : +- * Sort (126) - : : : : : : : : : : : : : : : +- * Project (125) - : : : : : : : : : : : : : : : +- * Filter (124) - : : : : : : : : : : : : : : : +- * HashAggregate (123) - : : : : : : : : : : : : : : : +- ReusedExchange (122) - : : : : : : : : : : : : : : +- ReusedExchange (129) - : : : : : : : : : : : : : +- ReusedExchange (132) - : : : : : : : : : : : : +- ReusedExchange (135) - : : : : : : : : : : : +- ReusedExchange (138) - : : : : : : : : : : +- ReusedExchange (141) - : : : : : : : : : +- ReusedExchange (144) - : : : : : : : : +- ReusedExchange (147) - : : : : : : : +- ReusedExchange (150) - : : : : : : +- ReusedExchange (153) - : : : : : +- ReusedExchange (156) - : : : : +- ReusedExchange (159) - : : : +- ReusedExchange (162) - : : +- ReusedExchange (165) - : +- ReusedExchange (168) - +- ReusedExchange (171) +* ColumnarToRow (176) ++- CometSort (175) + +- CometColumnarExchange (174) + +- RowToColumnar (173) + +- * Project (172) + +- * SortMergeJoin Inner (171) + :- * ColumnarToRow (101) + : +- CometSort (100) + : +- CometColumnarExchange (99) + : +- CometHashAggregate (98) + : +- CometHashAggregate (97) + : +- CometProject (96) + : +- CometBroadcastHashJoin (95) + : :- CometProject (90) + : : +- CometBroadcastHashJoin (89) + : : :- CometProject (87) + : : : +- CometBroadcastHashJoin (86) + : : : :- CometProject (82) + : : : : +- CometBroadcastHashJoin (81) + : : : : :- CometProject (79) + : : : : : +- CometBroadcastHashJoin (78) + : : : : : :- CometProject (74) + : : : : : : +- CometBroadcastHashJoin (73) + : : : : : : :- CometProject (71) + : : : : : : : +- CometBroadcastHashJoin (70) + : : : : : : : :- CometProject (66) + : : : : : : : : +- CometBroadcastHashJoin (65) + : : : : : : : : :- CometProject (61) + : : : : : : : : : +- CometBroadcastHashJoin (60) + : : : : : : : : : :- CometProject (58) + : : : : : : : : : : +- CometBroadcastHashJoin (57) + : : : : : : : : : : :- CometProject (53) + : : : : : : : : : : : +- CometBroadcastHashJoin (52) + : : : : : : : : : : : :- CometProject (50) + : : : : : : : : : : : : +- CometBroadcastHashJoin (49) + : : : : : : : : : : : : :- CometProject (45) + : : : : : : : : : : : : : +- CometBroadcastHashJoin (44) + : : : : : : : : : : : : : :- CometProject (40) + : : : : : : : : : : : : : : +- CometBroadcastHashJoin (39) + : : : : : : : : : : : : : : :- CometProject (35) + : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (34) + : : : : : : : : : : : : : : : :- CometProject (30) + : : : : : : : : : : : : : : : : +- CometSortMergeJoin (29) + : : : : : : : : : : : : : : : : :- CometSort (10) + : : : : : : : : : : : : : : : : : +- CometColumnarExchange (9) + : : : : : : : : : : : : : : : : : +- CometProject (8) + : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) + : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) + : : : : : : : : : : : : : : : : : : +- CometFilter (2) + : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : : : : : : : : : : : : +- CometProject (6) + : : : : : : : : : : : : : : : : : +- CometFilter (5) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : : : : : : : : : : : : : : : +- CometSort (28) + : : : : : : : : : : : : : : : : +- CometProject (27) + : : : : : : : : : : : : : : : : +- CometFilter (26) + : : : : : : : : : : : : : : : : +- CometHashAggregate (25) + : : : : : : : : : : : : : : : : +- CometColumnarExchange (24) + : : : : : : : : : : : : : : : : +- CometHashAggregate (23) + : : : : : : : : : : : : : : : : +- CometProject (22) + : : : : : : : : : : : : : : : : +- CometSortMergeJoin (21) + : : : : : : : : : : : : : : : : :- CometSort (15) + : : : : : : : : : : : : : : : : : +- CometColumnarExchange (14) + : : : : : : : : : : : : : : : : : +- CometProject (13) + : : : : : : : : : : : : : : : : : +- CometFilter (12) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (11) + : : : : : : : : : : : : : : : : +- CometSort (20) + : : : : : : : : : : : : : : : : +- CometColumnarExchange (19) + : : : : : : : : : : : : : : : : +- CometProject (18) + : : : : : : : : : : : : : : : : +- CometFilter (17) + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (16) + : : : : : : : : : : : : : : : +- CometBroadcastExchange (33) + : : : : : : : : : : : : : : : +- CometFilter (32) + : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (31) + : : : : : : : : : : : : : : +- CometBroadcastExchange (38) + : : : : : : : : : : : : : : +- CometFilter (37) + : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (36) + : : : : : : : : : : : : : +- CometBroadcastExchange (43) + : : : : : : : : : : : : : +- CometFilter (42) + : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (41) + : : : : : : : : : : : : +- CometBroadcastExchange (48) + : : : : : : : : : : : : +- CometFilter (47) + : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (46) + : : : : : : : : : : : +- ReusedExchange (51) + : : : : : : : : : : +- CometBroadcastExchange (56) + : : : : : : : : : : +- CometFilter (55) + : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (54) + : : : : : : : : : +- ReusedExchange (59) + : : : : : : : : +- CometBroadcastExchange (64) + : : : : : : : : +- CometFilter (63) + : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (62) + : : : : : : : +- CometBroadcastExchange (69) + : : : : : : : +- CometFilter (68) + : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (67) + : : : : : : +- ReusedExchange (72) + : : : : : +- CometBroadcastExchange (77) + : : : : : +- CometFilter (76) + : : : : : +- CometScan parquet spark_catalog.default.customer_address (75) + : : : : +- ReusedExchange (80) + : : : +- CometBroadcastExchange (85) + : : : +- CometFilter (84) + : : : +- CometScan parquet spark_catalog.default.income_band (83) + : : +- ReusedExchange (88) + : +- CometBroadcastExchange (94) + : +- CometProject (93) + : +- CometFilter (92) + : +- CometScan parquet spark_catalog.default.item (91) + +- * ColumnarToRow (170) + +- CometSort (169) + +- CometColumnarExchange (168) + +- CometHashAggregate (167) + +- CometHashAggregate (166) + +- CometProject (165) + +- CometBroadcastHashJoin (164) + :- CometProject (162) + : +- CometBroadcastHashJoin (161) + : :- CometProject (159) + : : +- CometBroadcastHashJoin (158) + : : :- CometProject (156) + : : : +- CometBroadcastHashJoin (155) + : : : :- CometProject (153) + : : : : +- CometBroadcastHashJoin (152) + : : : : :- CometProject (150) + : : : : : +- CometBroadcastHashJoin (149) + : : : : : :- CometProject (147) + : : : : : : +- CometBroadcastHashJoin (146) + : : : : : : :- CometProject (144) + : : : : : : : +- CometBroadcastHashJoin (143) + : : : : : : : :- CometProject (141) + : : : : : : : : +- CometBroadcastHashJoin (140) + : : : : : : : : :- CometProject (138) + : : : : : : : : : +- CometBroadcastHashJoin (137) + : : : : : : : : : :- CometProject (135) + : : : : : : : : : : +- CometBroadcastHashJoin (134) + : : : : : : : : : : :- CometProject (132) + : : : : : : : : : : : +- CometBroadcastHashJoin (131) + : : : : : : : : : : : :- CometProject (129) + : : : : : : : : : : : : +- CometBroadcastHashJoin (128) + : : : : : : : : : : : : :- CometProject (126) + : : : : : : : : : : : : : +- CometBroadcastHashJoin (125) + : : : : : : : : : : : : : :- CometProject (123) + : : : : : : : : : : : : : : +- CometBroadcastHashJoin (122) + : : : : : : : : : : : : : : :- CometProject (118) + : : : : : : : : : : : : : : : +- CometSortMergeJoin (117) + : : : : : : : : : : : : : : : :- CometSort (111) + : : : : : : : : : : : : : : : : +- CometColumnarExchange (110) + : : : : : : : : : : : : : : : : +- CometProject (109) + : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (108) + : : : : : : : : : : : : : : : : :- CometBroadcastExchange (104) + : : : : : : : : : : : : : : : : : +- CometFilter (103) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (102) + : : : : : : : : : : : : : : : : +- CometProject (107) + : : : : : : : : : : : : : : : : +- CometFilter (106) + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (105) + : : : : : : : : : : : : : : : +- CometSort (116) + : : : : : : : : : : : : : : : +- CometProject (115) + : : : : : : : : : : : : : : : +- CometFilter (114) + : : : : : : : : : : : : : : : +- CometHashAggregate (113) + : : : : : : : : : : : : : : : +- ReusedExchange (112) + : : : : : : : : : : : : : : +- CometBroadcastExchange (121) + : : : : : : : : : : : : : : +- CometFilter (120) + : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (119) + : : : : : : : : : : : : : +- ReusedExchange (124) + : : : : : : : : : : : : +- ReusedExchange (127) + : : : : : : : : : : : +- ReusedExchange (130) + : : : : : : : : : : +- ReusedExchange (133) + : : : : : : : : : +- ReusedExchange (136) + : : : : : : : : +- ReusedExchange (139) + : : : : : : : +- ReusedExchange (142) + : : : : : : +- ReusedExchange (145) + : : : : : +- ReusedExchange (148) + : : : : +- ReusedExchange (151) + : : : +- ReusedExchange (154) + : : +- ReusedExchange (157) + : +- ReusedExchange (160) + +- ReusedExchange (163) (1) Scan parquet spark_catalog.default.store_sales @@ -222,843 +217,793 @@ Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number# 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] Arguments: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -(9) ColumnarToRow [codegen id : 1] +(9) CometColumnarExchange Input [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(10) Exchange +(10) CometSort Input [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1 ASC NULLS FIRST] -(11) Sort [codegen id : 2] -Input [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 - -(12) Scan parquet spark_catalog.default.catalog_sales +(11) Scan parquet spark_catalog.default.catalog_sales Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_order_number)] ReadSchema: struct -(13) CometFilter +(12) CometFilter Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) -(14) CometProject +(13) CometProject Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -(15) ColumnarToRow [codegen id : 3] -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] - -(16) Exchange +(14) CometColumnarExchange Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(17) Sort [codegen id : 4] +(15) CometSort Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST], false, 0 +Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST] -(18) Scan parquet spark_catalog.default.catalog_returns +(16) Scan parquet spark_catalog.default.catalog_returns Output [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(19) CometFilter +(17) CometFilter Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) -(20) CometProject +(18) CometProject Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -(21) ColumnarToRow [codegen id : 5] +(19) CometColumnarExchange Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) Exchange +(20) CometSort Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST] -(23) Sort [codegen id : 6] -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST], false, 0 +(21) CometSortMergeJoin +Left output [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] +Right output [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number#22], Inner -(24) SortMergeJoin [codegen id : 7] -Left keys [2]: [cs_item_sk#17, cs_order_number#18] -Right keys [2]: [cr_item_sk#21, cr_order_number#22] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 7] -Output [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +(22) CometProject Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +Arguments: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -(26) HashAggregate [codegen id : 7] +(23) CometHashAggregate Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Keys [1]: [cs_item_sk#17] Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] -Aggregate Attributes [3]: [sum#27, sum#28, isEmpty#29] -Results [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] -(27) Exchange -Input [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(24) CometColumnarExchange +Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] +Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(28) HashAggregate [codegen id : 8] -Input [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] +(25) CometHashAggregate +Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] Keys [1]: [cs_item_sk#17] Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#19))#33, sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))#34] -Results [3]: [cs_item_sk#17, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#19))#33,17,2) AS sale#35, sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))#34 AS refund#36] -(29) Filter [codegen id : 8] -Input [3]: [cs_item_sk#17, sale#35, refund#36] -Condition : ((isnotnull(sale#35) AND isnotnull(refund#36)) AND (cast(sale#35 as decimal(21,2)) > (2 * refund#36))) +(26) CometFilter +Input [3]: [cs_item_sk#17, sale#30, refund#31] +Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) -(30) Project [codegen id : 8] -Output [1]: [cs_item_sk#17] -Input [3]: [cs_item_sk#17, sale#35, refund#36] +(27) CometProject +Input [3]: [cs_item_sk#17, sale#30, refund#31] +Arguments: [cs_item_sk#17], [cs_item_sk#17] -(31) Sort [codegen id : 8] +(28) CometSort Input [1]: [cs_item_sk#17] -Arguments: [cs_item_sk#17 ASC NULLS FIRST], false, 0 +Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] -(32) SortMergeJoin [codegen id : 24] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [cs_item_sk#17] -Join type: Inner -Join condition: None +(29) CometSortMergeJoin +Left output [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Right output [1]: [cs_item_sk#17] +Arguments: [ss_item_sk#1], [cs_item_sk#17], Inner -(33) Project [codegen id : 24] -Output [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +(30) CometProject Input [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#17] +Arguments: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] + +(31) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#32, d_year#33] +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 -(34) ReusedExchange [Reuses operator id: 185] -Output [2]: [d_date_sk#37, d_year#38] +(32) CometFilter +Input [2]: [d_date_sk#32, d_year#33] +Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(35) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#37] -Join type: Inner -Join condition: None +(33) CometBroadcastExchange +Input [2]: [d_date_sk#32, d_year#33] +Arguments: [d_date_sk#32, d_year#33] + +(34) CometBroadcastHashJoin +Left output [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Right output [2]: [d_date_sk#32, d_year#33] +Arguments: [ss_sold_date_sk#12], [d_date_sk#32], Inner, BuildRight -(36) Project [codegen id : 24] -Output [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38] -Input [13]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#37, d_year#38] +(35) CometProject +Input [13]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#32, d_year#33] +Arguments: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33], [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] -(37) Scan parquet spark_catalog.default.store -Output [3]: [s_store_sk#39, s_store_name#40, s_zip#41] +(36) Scan parquet spark_catalog.default.store +Output [3]: [s_store_sk#34, s_store_name#35, s_zip#36] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_zip)] ReadSchema: struct -(38) CometFilter -Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] -Condition : ((isnotnull(s_store_sk#39) AND isnotnull(s_store_name#40)) AND isnotnull(s_zip#41)) +(37) CometFilter +Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] +Condition : ((isnotnull(s_store_sk#34) AND isnotnull(s_store_name#35)) AND isnotnull(s_zip#36)) -(39) ColumnarToRow [codegen id : 10] -Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] +(38) CometBroadcastExchange +Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] +Arguments: [s_store_sk#34, s_store_name#35, s_zip#36] -(40) BroadcastExchange -Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +(39) CometBroadcastHashJoin +Left output [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] +Right output [3]: [s_store_sk#34, s_store_name#35, s_zip#36] +Arguments: [ss_store_sk#6], [s_store_sk#34], Inner, BuildRight -(41) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_store_sk#6] -Right keys [1]: [s_store_sk#39] -Join type: Inner -Join condition: None +(40) 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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_sk#34, s_store_name#35, s_zip#36] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36] -(42) Project [codegen id : 24] -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41] -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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_sk#39, s_store_name#40, s_zip#41] - -(43) Scan parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] +(41) Scan parquet spark_catalog.default.customer +Output [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(44) CometFilter -Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] -Condition : (((((isnotnull(c_customer_sk#42) AND isnotnull(c_first_sales_date_sk#47)) AND isnotnull(c_first_shipto_date_sk#46)) AND isnotnull(c_current_cdemo_sk#43)) AND isnotnull(c_current_hdemo_sk#44)) AND isnotnull(c_current_addr_sk#45)) - -(45) ColumnarToRow [codegen id : 11] -Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] +(42) CometFilter +Input [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +Condition : (((((isnotnull(c_customer_sk#37) AND isnotnull(c_first_sales_date_sk#42)) AND isnotnull(c_first_shipto_date_sk#41)) AND isnotnull(c_current_cdemo_sk#38)) AND isnotnull(c_current_hdemo_sk#39)) AND isnotnull(c_current_addr_sk#40)) -(46) BroadcastExchange -Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +(43) CometBroadcastExchange +Input [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +Arguments: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] -(47) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#42] -Join type: Inner -Join condition: None +(44) 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_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36] +Right output [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +Arguments: [ss_customer_sk#2], [c_customer_sk#37], Inner, BuildRight -(48) Project [codegen id : 24] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] +(45) CometProject +Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] -(49) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#48, d_year#49] +(46) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#43, d_year#44] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct -(50) CometFilter -Input [2]: [d_date_sk#48, d_year#49] -Condition : isnotnull(d_date_sk#48) +(47) CometFilter +Input [2]: [d_date_sk#43, d_year#44] +Condition : isnotnull(d_date_sk#43) -(51) ColumnarToRow [codegen id : 12] -Input [2]: [d_date_sk#48, d_year#49] +(48) CometBroadcastExchange +Input [2]: [d_date_sk#43, d_year#44] +Arguments: [d_date_sk#43, d_year#44] -(52) BroadcastExchange -Input [2]: [d_date_sk#48, d_year#49] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] - -(53) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [c_first_sales_date_sk#47] -Right keys [1]: [d_date_sk#48] -Join type: Inner -Join condition: None +(49) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +Right output [2]: [d_date_sk#43, d_year#44] +Arguments: [c_first_sales_date_sk#42], [d_date_sk#43], Inner, BuildRight -(54) Project [codegen id : 24] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, d_year#49] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47, d_date_sk#48, d_year#49] +(50) CometProject +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42, d_date_sk#43, d_year#44] +Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, d_year#44], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, d_year#44] -(55) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#50, d_year#51] +(51) ReusedExchange [Reuses operator id: 48] +Output [2]: [d_date_sk#45, d_year#46] -(56) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [c_first_shipto_date_sk#46] -Right keys [1]: [d_date_sk#50] -Join type: Inner -Join condition: None +(52) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, d_year#44] +Right output [2]: [d_date_sk#45, d_year#46] +Arguments: [c_first_shipto_date_sk#41], [d_date_sk#45], Inner, BuildRight -(57) Project [codegen id : 24] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, d_year#49, d_date_sk#50, d_year#51] +(53) CometProject +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, d_year#44, d_date_sk#45, d_year#46] +Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] -(58) Scan parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#52, cd_marital_status#53] +(54) Scan parquet spark_catalog.default.customer_demographics +Output [2]: [cd_demo_sk#47, cd_marital_status#48] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status)] ReadSchema: struct -(59) CometFilter -Input [2]: [cd_demo_sk#52, cd_marital_status#53] -Condition : (isnotnull(cd_demo_sk#52) AND isnotnull(cd_marital_status#53)) - -(60) ColumnarToRow [codegen id : 14] -Input [2]: [cd_demo_sk#52, cd_marital_status#53] +(55) CometFilter +Input [2]: [cd_demo_sk#47, cd_marital_status#48] +Condition : (isnotnull(cd_demo_sk#47) AND isnotnull(cd_marital_status#48)) -(61) BroadcastExchange -Input [2]: [cd_demo_sk#52, cd_marital_status#53] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +(56) CometBroadcastExchange +Input [2]: [cd_demo_sk#47, cd_marital_status#48] +Arguments: [cd_demo_sk#47, cd_marital_status#48] -(62) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#52] -Join type: Inner -Join condition: None +(57) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] +Right output [2]: [cd_demo_sk#47, cd_marital_status#48] +Arguments: [ss_cdemo_sk#3], [cd_demo_sk#47], Inner, BuildRight -(63) Project [codegen id : 24] -Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_marital_status#53] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_demo_sk#52, cd_marital_status#53] +(58) CometProject +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_demo_sk#47, cd_marital_status#48] +Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_marital_status#48], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_marital_status#48] -(64) ReusedExchange [Reuses operator id: 61] -Output [2]: [cd_demo_sk#54, cd_marital_status#55] +(59) ReusedExchange [Reuses operator id: 56] +Output [2]: [cd_demo_sk#49, cd_marital_status#50] -(65) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [c_current_cdemo_sk#43] -Right keys [1]: [cd_demo_sk#54] -Join type: Inner -Join condition: NOT (cd_marital_status#53 = cd_marital_status#55) +(60) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_marital_status#48] +Right output [2]: [cd_demo_sk#49, cd_marital_status#50] +Arguments: [c_current_cdemo_sk#38], [cd_demo_sk#49], Inner, NOT (cd_marital_status#48 = cd_marital_status#50), BuildRight -(66) Project [codegen id : 24] -Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_marital_status#53, cd_demo_sk#54, cd_marital_status#55] +(61) CometProject +Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_marital_status#48, cd_demo_sk#49, cd_marital_status#50] +Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] -(67) Scan parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#56] +(62) Scan parquet spark_catalog.default.promotion +Output [1]: [p_promo_sk#51] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_promo_sk)] ReadSchema: struct -(68) CometFilter -Input [1]: [p_promo_sk#56] -Condition : isnotnull(p_promo_sk#56) +(63) CometFilter +Input [1]: [p_promo_sk#51] +Condition : isnotnull(p_promo_sk#51) -(69) ColumnarToRow [codegen id : 16] -Input [1]: [p_promo_sk#56] +(64) CometBroadcastExchange +Input [1]: [p_promo_sk#51] +Arguments: [p_promo_sk#51] -(70) BroadcastExchange -Input [1]: [p_promo_sk#56] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] +(65) CometBroadcastHashJoin +Left output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] +Right output [1]: [p_promo_sk#51] +Arguments: [ss_promo_sk#7], [p_promo_sk#51], Inner, BuildRight -(71) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_promo_sk#7] -Right keys [1]: [p_promo_sk#56] -Join type: Inner -Join condition: None - -(72) Project [codegen id : 24] -Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, p_promo_sk#56] +(66) CometProject +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, p_promo_sk#51] +Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] -(73) Scan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#57, hd_income_band_sk#58] +(67) Scan parquet spark_catalog.default.household_demographics +Output [2]: [hd_demo_sk#52, hd_income_band_sk#53] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] ReadSchema: struct -(74) CometFilter -Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] -Condition : (isnotnull(hd_demo_sk#57) AND isnotnull(hd_income_band_sk#58)) - -(75) ColumnarToRow [codegen id : 17] -Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] +(68) CometFilter +Input [2]: [hd_demo_sk#52, hd_income_band_sk#53] +Condition : (isnotnull(hd_demo_sk#52) AND isnotnull(hd_income_band_sk#53)) -(76) BroadcastExchange -Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] +(69) CometBroadcastExchange +Input [2]: [hd_demo_sk#52, hd_income_band_sk#53] +Arguments: [hd_demo_sk#52, hd_income_band_sk#53] -(77) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_hdemo_sk#4] -Right keys [1]: [hd_demo_sk#57] -Join type: Inner -Join condition: None +(70) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] +Right output [2]: [hd_demo_sk#52, hd_income_band_sk#53] +Arguments: [ss_hdemo_sk#4], [hd_demo_sk#52], Inner, BuildRight -(78) Project [codegen id : 24] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_demo_sk#57, hd_income_band_sk#58] +(71) CometProject +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_demo_sk#52, hd_income_band_sk#53] +Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53] -(79) ReusedExchange [Reuses operator id: 76] -Output [2]: [hd_demo_sk#59, hd_income_band_sk#60] +(72) ReusedExchange [Reuses operator id: 69] +Output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -(80) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [c_current_hdemo_sk#44] -Right keys [1]: [hd_demo_sk#59] -Join type: Inner -Join condition: None +(73) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53] +Right output [2]: [hd_demo_sk#54, hd_income_band_sk#55] +Arguments: [c_current_hdemo_sk#39], [hd_demo_sk#54], Inner, BuildRight -(81) Project [codegen id : 24] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60] -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_demo_sk#59, hd_income_band_sk#60] +(74) CometProject +Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_demo_sk#54, hd_income_band_sk#55] +Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55] -(82) Scan parquet spark_catalog.default.customer_address -Output [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +(75) Scan parquet spark_catalog.default.customer_address +Output [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(83) CometFilter -Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -Condition : isnotnull(ca_address_sk#61) +(76) CometFilter +Input [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +Condition : isnotnull(ca_address_sk#56) -(84) ColumnarToRow [codegen id : 19] -Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +(77) CometBroadcastExchange +Input [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +Arguments: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] -(85) BroadcastExchange -Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=11] +(78) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55] +Right output [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +Arguments: [ss_addr_sk#5], [ca_address_sk#56], Inner, BuildRight -(86) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_addr_sk#5] -Right keys [1]: [ca_address_sk#61] -Join type: Inner -Join condition: None - -(87) Project [codegen id : 24] -Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +(79) CometProject +Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] -(88) ReusedExchange [Reuses operator id: 85] -Output [5]: [ca_address_sk#66, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] +(80) ReusedExchange [Reuses operator id: 77] +Output [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -(89) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [c_current_addr_sk#45] -Right keys [1]: [ca_address_sk#66] -Join type: Inner -Join condition: None +(81) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +Right output [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Arguments: [c_current_addr_sk#40], [ca_address_sk#61], Inner, BuildRight -(90) Project [codegen id : 24] -Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_address_sk#66, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] +(82) CometProject +Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -(91) Scan parquet spark_catalog.default.income_band -Output [1]: [ib_income_band_sk#71] +(83) Scan parquet spark_catalog.default.income_band +Output [1]: [ib_income_band_sk#66] Batched: true Location [not included in comparison]/{warehouse_dir}/income_band] PushedFilters: [IsNotNull(ib_income_band_sk)] ReadSchema: struct -(92) CometFilter -Input [1]: [ib_income_band_sk#71] -Condition : isnotnull(ib_income_band_sk#71) +(84) CometFilter +Input [1]: [ib_income_band_sk#66] +Condition : isnotnull(ib_income_band_sk#66) -(93) ColumnarToRow [codegen id : 21] -Input [1]: [ib_income_band_sk#71] +(85) CometBroadcastExchange +Input [1]: [ib_income_band_sk#66] +Arguments: [ib_income_band_sk#66] -(94) BroadcastExchange -Input [1]: [ib_income_band_sk#71] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] - -(95) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [hd_income_band_sk#58] -Right keys [1]: [ib_income_band_sk#71] -Join type: Inner -Join condition: None +(86) CometBroadcastHashJoin +Left output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Right output [1]: [ib_income_band_sk#66] +Arguments: [hd_income_band_sk#53], [ib_income_band_sk#66], Inner, BuildRight -(96) Project [codegen id : 24] -Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, ib_income_band_sk#71] +(87) CometProject +Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ib_income_band_sk#66] +Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -(97) ReusedExchange [Reuses operator id: 94] -Output [1]: [ib_income_band_sk#72] +(88) ReusedExchange [Reuses operator id: 85] +Output [1]: [ib_income_band_sk#67] -(98) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [hd_income_band_sk#60] -Right keys [1]: [ib_income_band_sk#72] -Join type: Inner -Join condition: None +(89) CometBroadcastHashJoin +Left output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Right output [1]: [ib_income_band_sk#67] +Arguments: [hd_income_band_sk#55], [ib_income_band_sk#67], Inner, BuildRight -(99) Project [codegen id : 24] -Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, ib_income_band_sk#72] +(90) CometProject +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ib_income_band_sk#67] +Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -(100) Scan parquet spark_catalog.default.item -Output [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] +(91) Scan parquet spark_catalog.default.item +Output [4]: [i_item_sk#68, i_current_price#69, i_color#70, i_product_name#71] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), In(i_color, [burlywood ,floral ,indian ,medium ,purple ,spring ]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] ReadSchema: struct -(101) CometFilter -Input [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] -Condition : ((((((isnotnull(i_current_price#74) AND i_color#75 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#74 >= 64.00)) AND (i_current_price#74 <= 74.00)) AND (i_current_price#74 >= 65.00)) AND (i_current_price#74 <= 79.00)) AND isnotnull(i_item_sk#73)) - -(102) CometProject -Input [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] -Arguments: [i_item_sk#73, i_product_name#76], [i_item_sk#73, i_product_name#76] +(92) CometFilter +Input [4]: [i_item_sk#68, i_current_price#69, i_color#70, i_product_name#71] +Condition : ((((((isnotnull(i_current_price#69) AND i_color#70 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#69 >= 64.00)) AND (i_current_price#69 <= 74.00)) AND (i_current_price#69 >= 65.00)) AND (i_current_price#69 <= 79.00)) AND isnotnull(i_item_sk#68)) -(103) ColumnarToRow [codegen id : 23] -Input [2]: [i_item_sk#73, i_product_name#76] +(93) CometProject +Input [4]: [i_item_sk#68, i_current_price#69, i_color#70, i_product_name#71] +Arguments: [i_item_sk#68, i_product_name#71], [i_item_sk#68, i_product_name#71] -(104) BroadcastExchange -Input [2]: [i_item_sk#73, i_product_name#76] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] +(94) CometBroadcastExchange +Input [2]: [i_item_sk#68, i_product_name#71] +Arguments: [i_item_sk#68, i_product_name#71] -(105) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#73] -Join type: Inner -Join condition: None +(95) CometBroadcastHashJoin +Left output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Right output [2]: [i_item_sk#68, i_product_name#71] +Arguments: [ss_item_sk#1], [i_item_sk#68], Inner, BuildRight -(106) Project [codegen id : 24] -Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, d_year#49, d_year#51, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] +(96) CometProject +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, i_item_sk#68, i_product_name#71] +Arguments: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#44, d_year#46, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, i_item_sk#68, i_product_name#71], [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#44, d_year#46, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, i_item_sk#68, i_product_name#71] -(107) HashAggregate [codegen id : 24] -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, d_year#49, d_year#51, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51] +(97) CometHashAggregate +Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#44, d_year#46, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, i_item_sk#68, i_product_name#71] +Keys [15]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46] Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count#77, sum#78, sum#79, sum#80] -Results [19]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51, count#81, sum#82, sum#83, sum#84] -(108) HashAggregate [codegen id : 24] -Input [19]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51, count#81, sum#82, sum#83, sum#84] -Keys [15]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51] +(98) CometHashAggregate +Input [19]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46, count#72, sum#73, sum#74, sum#75] +Keys [15]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46] Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count(1)#85, sum(UnscaledValue(ss_wholesale_cost#9))#86, sum(UnscaledValue(ss_list_price#10))#87, sum(UnscaledValue(ss_coupon_amt#11))#88] -Results [17]: [i_product_name#76 AS product_name#89, i_item_sk#73 AS item_sk#90, s_store_name#40 AS store_name#91, s_zip#41 AS store_zip#92, ca_street_number#62 AS b_street_number#93, ca_street_name#63 AS b_streen_name#94, ca_city#64 AS b_city#95, ca_zip#65 AS b_zip#96, ca_street_number#67 AS c_street_number#97, ca_street_name#68 AS c_street_name#98, ca_city#69 AS c_city#99, ca_zip#70 AS c_zip#100, d_year#38 AS syear#101, count(1)#85 AS cnt#102, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#86,17,2) AS s1#103, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#87,17,2) AS s2#104, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#88,17,2) AS s3#105] -(109) Exchange -Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] -Arguments: hashpartitioning(item_sk#90, store_name#91, store_zip#92, 5), ENSURE_REQUIREMENTS, [plan_id=14] +(99) CometColumnarExchange +Input [17]: [product_name#76, item_sk#77, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92] +Arguments: hashpartitioning(item_sk#77, store_name#78, store_zip#79, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(100) CometSort +Input [17]: [product_name#76, item_sk#77, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92] +Arguments: [product_name#76, item_sk#77, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92], [item_sk#77 ASC NULLS FIRST, store_name#78 ASC NULLS FIRST, store_zip#79 ASC NULLS FIRST] -(110) Sort [codegen id : 25] -Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] -Arguments: [item_sk#90 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, store_zip#92 ASC NULLS FIRST], false, 0 +(101) ColumnarToRow [codegen id : 1] +Input [17]: [product_name#76, item_sk#77, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92] -(111) Scan parquet spark_catalog.default.store_sales -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] +(102) Scan parquet spark_catalog.default.store_sales +Output [12]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_ticket_number#100, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#117), dynamicpruningexpression(ss_sold_date_sk#117 IN dynamicpruning#118)] +PartitionFilters: [isnotnull(ss_sold_date_sk#104), dynamicpruningexpression(ss_sold_date_sk#104 IN dynamicpruning#105)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct -(112) CometFilter -Input [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] -Condition : (((((((isnotnull(ss_item_sk#106) AND isnotnull(ss_ticket_number#113)) AND isnotnull(ss_store_sk#111)) AND isnotnull(ss_customer_sk#107)) AND isnotnull(ss_cdemo_sk#108)) AND isnotnull(ss_promo_sk#112)) AND isnotnull(ss_hdemo_sk#109)) AND isnotnull(ss_addr_sk#110)) +(103) CometFilter +Input [12]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_ticket_number#100, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] +Condition : (((((((isnotnull(ss_item_sk#93) AND isnotnull(ss_ticket_number#100)) AND isnotnull(ss_store_sk#98)) AND isnotnull(ss_customer_sk#94)) AND isnotnull(ss_cdemo_sk#95)) AND isnotnull(ss_promo_sk#99)) AND isnotnull(ss_hdemo_sk#96)) AND isnotnull(ss_addr_sk#97)) -(113) CometBroadcastExchange -Input [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] -Arguments: [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] +(104) CometBroadcastExchange +Input [12]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_ticket_number#100, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] +Arguments: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_ticket_number#100, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] -(114) Scan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] +(105) Scan parquet spark_catalog.default.store_returns +Output [3]: [sr_item_sk#106, sr_ticket_number#107, sr_returned_date_sk#108] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] ReadSchema: struct -(115) CometFilter -Input [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] -Condition : (isnotnull(sr_item_sk#119) AND isnotnull(sr_ticket_number#120)) +(106) CometFilter +Input [3]: [sr_item_sk#106, sr_ticket_number#107, sr_returned_date_sk#108] +Condition : (isnotnull(sr_item_sk#106) AND isnotnull(sr_ticket_number#107)) -(116) CometProject -Input [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] -Arguments: [sr_item_sk#119, sr_ticket_number#120], [sr_item_sk#119, sr_ticket_number#120] +(107) CometProject +Input [3]: [sr_item_sk#106, sr_ticket_number#107, sr_returned_date_sk#108] +Arguments: [sr_item_sk#106, sr_ticket_number#107], [sr_item_sk#106, sr_ticket_number#107] -(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, BuildLeft +(108) CometBroadcastHashJoin +Left output [12]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_ticket_number#100, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] +Right output [2]: [sr_item_sk#106, sr_ticket_number#107] +Arguments: [ss_item_sk#93, ss_ticket_number#100], [sr_item_sk#106, sr_ticket_number#107], 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] -Arguments: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117], [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +(109) CometProject +Input [14]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_ticket_number#100, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104, sr_item_sk#106, sr_ticket_number#107] +Arguments: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104], [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] -(119) ColumnarToRow [codegen id : 26] -Input [11]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +(110) CometColumnarExchange +Input [11]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] +Arguments: hashpartitioning(ss_item_sk#93, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(120) Exchange -Input [11]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -Arguments: hashpartitioning(ss_item_sk#106, 5), ENSURE_REQUIREMENTS, [plan_id=15] +(111) CometSort +Input [11]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] +Arguments: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104], [ss_item_sk#93 ASC NULLS FIRST] -(121) Sort [codegen id : 27] -Input [11]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -Arguments: [ss_item_sk#106 ASC NULLS FIRST], false, 0 +(112) ReusedExchange [Reuses operator id: 24] +Output [4]: [cs_item_sk#109, sum#110, sum#111, isEmpty#112] -(122) ReusedExchange [Reuses operator id: 27] -Output [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] +(113) CometHashAggregate +Input [4]: [cs_item_sk#109, sum#110, sum#111, isEmpty#112] +Keys [1]: [cs_item_sk#109] +Functions [2]: [sum(UnscaledValue(cs_ext_list_price#113)), sum(((cr_refunded_cash#114 + cr_reversed_charge#115) + cr_store_credit#116))] -(123) HashAggregate [codegen id : 33] -Input [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] -Keys [1]: [cs_item_sk#122] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#126)), sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#126))#33, sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))#34] -Results [3]: [cs_item_sk#122, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#126))#33,17,2) AS sale#35, sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))#34 AS refund#36] +(114) CometFilter +Input [3]: [cs_item_sk#109, sale#30, refund#31] +Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) -(124) Filter [codegen id : 33] -Input [3]: [cs_item_sk#122, sale#35, refund#36] -Condition : ((isnotnull(sale#35) AND isnotnull(refund#36)) AND (cast(sale#35 as decimal(21,2)) > (2 * refund#36))) +(115) CometProject +Input [3]: [cs_item_sk#109, sale#30, refund#31] +Arguments: [cs_item_sk#109], [cs_item_sk#109] -(125) Project [codegen id : 33] -Output [1]: [cs_item_sk#122] -Input [3]: [cs_item_sk#122, sale#35, refund#36] +(116) CometSort +Input [1]: [cs_item_sk#109] +Arguments: [cs_item_sk#109], [cs_item_sk#109 ASC NULLS FIRST] -(126) Sort [codegen id : 33] -Input [1]: [cs_item_sk#122] -Arguments: [cs_item_sk#122 ASC NULLS FIRST], false, 0 +(117) CometSortMergeJoin +Left output [11]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] +Right output [1]: [cs_item_sk#109] +Arguments: [ss_item_sk#93], [cs_item_sk#109], Inner -(127) SortMergeJoin [codegen id : 49] -Left keys [1]: [ss_item_sk#106] -Right keys [1]: [cs_item_sk#122] -Join type: Inner -Join condition: None +(118) CometProject +Input [12]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104, cs_item_sk#109] +Arguments: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104], [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] -(128) Project [codegen id : 49] -Output [11]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -Input [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, cs_item_sk#122] +(119) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#117, d_year#118] +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 -(129) ReusedExchange [Reuses operator id: 189] -Output [2]: [d_date_sk#130, d_year#131] +(120) CometFilter +Input [2]: [d_date_sk#117, d_year#118] +Condition : ((isnotnull(d_year#118) AND (d_year#118 = 2000)) AND isnotnull(d_date_sk#117)) -(130) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [ss_sold_date_sk#117] -Right keys [1]: [d_date_sk#130] -Join type: Inner -Join condition: None +(121) CometBroadcastExchange +Input [2]: [d_date_sk#117, d_year#118] +Arguments: [d_date_sk#117, d_year#118] -(131) Project [codegen id : 49] -Output [11]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131] -Input [13]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, d_date_sk#130, d_year#131] +(122) CometBroadcastHashJoin +Left output [11]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] +Right output [2]: [d_date_sk#117, d_year#118] +Arguments: [ss_sold_date_sk#104], [d_date_sk#117], Inner, BuildRight -(132) ReusedExchange [Reuses operator id: 40] -Output [3]: [s_store_sk#132, s_store_name#133, s_zip#134] +(123) CometProject +Input [13]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104, d_date_sk#117, d_year#118] +Arguments: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118], [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118] -(133) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [ss_store_sk#111] -Right keys [1]: [s_store_sk#132] -Join type: Inner -Join condition: None +(124) ReusedExchange [Reuses operator id: 38] +Output [3]: [s_store_sk#119, s_store_name#120, s_zip#121] -(134) Project [codegen id : 49] -Output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134] -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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_sk#132, s_store_name#133, s_zip#134] +(125) CometBroadcastHashJoin +Left output [11]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118] +Right output [3]: [s_store_sk#119, s_store_name#120, s_zip#121] +Arguments: [ss_store_sk#98], [s_store_sk#119], Inner, BuildRight -(135) ReusedExchange [Reuses operator id: 46] -Output [6]: [c_customer_sk#135, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, c_first_sales_date_sk#140] +(126) CometProject +Input [14]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_sk#119, s_store_name#120, s_zip#121] +Arguments: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121], [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121] -(136) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [ss_customer_sk#107] -Right keys [1]: [c_customer_sk#135] -Join type: Inner -Join condition: None +(127) ReusedExchange [Reuses operator id: 43] +Output [6]: [c_customer_sk#122, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, c_first_sales_date_sk#127] -(137) Project [codegen id : 49] -Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, c_first_sales_date_sk#140] -Input [18]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_customer_sk#135, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, c_first_sales_date_sk#140] +(128) CometBroadcastHashJoin +Left output [12]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121] +Right output [6]: [c_customer_sk#122, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, c_first_sales_date_sk#127] +Arguments: [ss_customer_sk#94], [c_customer_sk#122], Inner, BuildRight -(138) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#141, d_year#142] +(129) CometProject +Input [18]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_customer_sk#122, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, c_first_sales_date_sk#127] +Arguments: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, c_first_sales_date_sk#127], [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, c_first_sales_date_sk#127] -(139) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [c_first_sales_date_sk#140] -Right keys [1]: [d_date_sk#141] -Join type: Inner -Join condition: None +(130) ReusedExchange [Reuses operator id: 48] +Output [2]: [d_date_sk#128, d_year#129] -(140) Project [codegen id : 49] -Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, d_year#142] -Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, c_first_sales_date_sk#140, d_date_sk#141, d_year#142] +(131) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, c_first_sales_date_sk#127] +Right output [2]: [d_date_sk#128, d_year#129] +Arguments: [c_first_sales_date_sk#127], [d_date_sk#128], Inner, BuildRight -(141) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#143, d_year#144] +(132) CometProject +Input [18]: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, c_first_sales_date_sk#127, d_date_sk#128, d_year#129] +Arguments: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, d_year#129], [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, d_year#129] -(142) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [c_first_shipto_date_sk#139] -Right keys [1]: [d_date_sk#143] -Join type: Inner -Join condition: None +(133) ReusedExchange [Reuses operator id: 48] +Output [2]: [d_date_sk#130, d_year#131] -(143) Project [codegen id : 49] -Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144] -Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, d_year#142, d_date_sk#143, d_year#144] +(134) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, d_year#129] +Right output [2]: [d_date_sk#130, d_year#131] +Arguments: [c_first_shipto_date_sk#126], [d_date_sk#130], Inner, BuildRight -(144) ReusedExchange [Reuses operator id: 61] -Output [2]: [cd_demo_sk#145, cd_marital_status#146] +(135) CometProject +Input [18]: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, d_year#129, d_date_sk#130, d_year#131] +Arguments: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131], [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131] -(145) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [ss_cdemo_sk#108] -Right keys [1]: [cd_demo_sk#145] -Join type: Inner -Join condition: None +(136) ReusedExchange [Reuses operator id: 56] +Output [2]: [cd_demo_sk#132, cd_marital_status#133] -(146) Project [codegen id : 49] -Output [16]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, cd_marital_status#146] -Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, cd_demo_sk#145, cd_marital_status#146] +(137) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131] +Right output [2]: [cd_demo_sk#132, cd_marital_status#133] +Arguments: [ss_cdemo_sk#95], [cd_demo_sk#132], Inner, BuildRight -(147) ReusedExchange [Reuses operator id: 61] -Output [2]: [cd_demo_sk#147, cd_marital_status#148] +(138) CometProject +Input [18]: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, cd_demo_sk#132, cd_marital_status#133] +Arguments: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, cd_marital_status#133], [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, cd_marital_status#133] -(148) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [c_current_cdemo_sk#136] -Right keys [1]: [cd_demo_sk#147] -Join type: Inner -Join condition: NOT (cd_marital_status#146 = cd_marital_status#148) +(139) ReusedExchange [Reuses operator id: 56] +Output [2]: [cd_demo_sk#134, cd_marital_status#135] -(149) Project [codegen id : 49] -Output [14]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144] -Input [18]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, cd_marital_status#146, cd_demo_sk#147, cd_marital_status#148] +(140) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, cd_marital_status#133] +Right output [2]: [cd_demo_sk#134, cd_marital_status#135] +Arguments: [c_current_cdemo_sk#123], [cd_demo_sk#134], Inner, NOT (cd_marital_status#133 = cd_marital_status#135), BuildRight -(150) ReusedExchange [Reuses operator id: 70] -Output [1]: [p_promo_sk#149] +(141) CometProject +Input [18]: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, cd_marital_status#133, cd_demo_sk#134, cd_marital_status#135] +Arguments: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131], [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131] -(151) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [ss_promo_sk#112] -Right keys [1]: [p_promo_sk#149] -Join type: Inner -Join condition: None +(142) ReusedExchange [Reuses operator id: 64] +Output [1]: [p_promo_sk#136] -(152) Project [codegen id : 49] -Output [13]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144] -Input [15]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, p_promo_sk#149] +(143) CometBroadcastHashJoin +Left output [14]: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131] +Right output [1]: [p_promo_sk#136] +Arguments: [ss_promo_sk#99], [p_promo_sk#136], Inner, BuildRight -(153) ReusedExchange [Reuses operator id: 76] -Output [2]: [hd_demo_sk#150, hd_income_band_sk#151] +(144) CometProject +Input [15]: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, p_promo_sk#136] +Arguments: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131], [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131] -(154) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [ss_hdemo_sk#109] -Right keys [1]: [hd_demo_sk#150] -Join type: Inner -Join condition: None +(145) ReusedExchange [Reuses operator id: 69] +Output [2]: [hd_demo_sk#137, hd_income_band_sk#138] -(155) Project [codegen id : 49] -Output [13]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151] -Input [15]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, hd_demo_sk#150, hd_income_band_sk#151] +(146) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131] +Right output [2]: [hd_demo_sk#137, hd_income_band_sk#138] +Arguments: [ss_hdemo_sk#96], [hd_demo_sk#137], Inner, BuildRight -(156) ReusedExchange [Reuses operator id: 76] -Output [2]: [hd_demo_sk#152, hd_income_band_sk#153] +(147) CometProject +Input [15]: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, hd_demo_sk#137, hd_income_band_sk#138] +Arguments: [ss_item_sk#93, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138], [ss_item_sk#93, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138] -(157) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [c_current_hdemo_sk#137] -Right keys [1]: [hd_demo_sk#152] -Join type: Inner -Join condition: None +(148) ReusedExchange [Reuses operator id: 69] +Output [2]: [hd_demo_sk#139, hd_income_band_sk#140] -(158) Project [codegen id : 49] -Output [13]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153] -Input [15]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_demo_sk#152, hd_income_band_sk#153] +(149) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#93, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138] +Right output [2]: [hd_demo_sk#139, hd_income_band_sk#140] +Arguments: [c_current_hdemo_sk#124], [hd_demo_sk#139], Inner, BuildRight -(159) ReusedExchange [Reuses operator id: 85] -Output [5]: [ca_address_sk#154, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] +(150) CometProject +Input [15]: [ss_item_sk#93, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_demo_sk#139, hd_income_band_sk#140] +Arguments: [ss_item_sk#93, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140], [ss_item_sk#93, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140] -(160) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [ss_addr_sk#110] -Right keys [1]: [ca_address_sk#154] -Join type: Inner -Join condition: None +(151) ReusedExchange [Reuses operator id: 77] +Output [5]: [ca_address_sk#141, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] -(161) Project [codegen id : 49] -Output [16]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] -Input [18]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_address_sk#154, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] +(152) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#93, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140] +Right output [5]: [ca_address_sk#141, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Arguments: [ss_addr_sk#97], [ca_address_sk#141], Inner, BuildRight -(162) ReusedExchange [Reuses operator id: 85] -Output [5]: [ca_address_sk#159, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] +(153) CometProject +Input [18]: [ss_item_sk#93, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_address_sk#141, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Arguments: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145], [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] -(163) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [c_current_addr_sk#138] -Right keys [1]: [ca_address_sk#159] -Join type: Inner -Join condition: None +(154) ReusedExchange [Reuses operator id: 77] +Output [5]: [ca_address_sk#146, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] -(164) Project [codegen id : 49] -Output [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] -Input [21]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_address_sk#159, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] +(155) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Right output [5]: [ca_address_sk#146, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] +Arguments: [c_current_addr_sk#125], [ca_address_sk#146], Inner, BuildRight -(165) ReusedExchange [Reuses operator id: 94] -Output [1]: [ib_income_band_sk#164] +(156) CometProject +Input [21]: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_address_sk#146, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] +Arguments: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150], [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] -(166) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [hd_income_band_sk#151] -Right keys [1]: [ib_income_band_sk#164] -Join type: Inner -Join condition: None +(157) ReusedExchange [Reuses operator id: 85] +Output [1]: [ib_income_band_sk#151] -(167) Project [codegen id : 49] -Output [18]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] -Input [20]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, ib_income_band_sk#164] +(158) CometBroadcastHashJoin +Left output [19]: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] +Right output [1]: [ib_income_band_sk#151] +Arguments: [hd_income_band_sk#138], [ib_income_band_sk#151], Inner, BuildRight -(168) ReusedExchange [Reuses operator id: 94] -Output [1]: [ib_income_band_sk#165] +(159) CometProject +Input [20]: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, ib_income_band_sk#151] +Arguments: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150], [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] -(169) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [hd_income_band_sk#153] -Right keys [1]: [ib_income_band_sk#165] -Join type: Inner -Join condition: None +(160) ReusedExchange [Reuses operator id: 85] +Output [1]: [ib_income_band_sk#152] -(170) Project [codegen id : 49] -Output [17]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] -Input [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, ib_income_band_sk#165] +(161) CometBroadcastHashJoin +Left output [18]: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] +Right output [1]: [ib_income_band_sk#152] +Arguments: [hd_income_band_sk#140], [ib_income_band_sk#152], Inner, BuildRight -(171) ReusedExchange [Reuses operator id: 104] -Output [2]: [i_item_sk#166, i_product_name#167] +(162) CometProject +Input [19]: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, ib_income_band_sk#152] +Arguments: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150], [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] -(172) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [ss_item_sk#106] -Right keys [1]: [i_item_sk#166] -Join type: Inner -Join condition: None - -(173) Project [codegen id : 49] -Output [18]: [ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, d_year#142, d_year#144, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, i_item_sk#166, i_product_name#167] -Input [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, i_item_sk#166, i_product_name#167] - -(174) HashAggregate [codegen id : 49] -Input [18]: [ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, d_year#142, d_year#144, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, i_item_sk#166, i_product_name#167] -Keys [15]: [i_product_name#167, i_item_sk#166, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, d_year#131, d_year#142, d_year#144] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#114)), partial_sum(UnscaledValue(ss_list_price#115)), partial_sum(UnscaledValue(ss_coupon_amt#116))] -Aggregate Attributes [4]: [count#77, sum#168, sum#169, sum#170] -Results [19]: [i_product_name#167, i_item_sk#166, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, d_year#131, d_year#142, d_year#144, count#81, sum#171, sum#172, sum#173] - -(175) HashAggregate [codegen id : 49] -Input [19]: [i_product_name#167, i_item_sk#166, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, d_year#131, d_year#142, d_year#144, count#81, sum#171, sum#172, sum#173] -Keys [15]: [i_product_name#167, i_item_sk#166, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, d_year#131, d_year#142, d_year#144] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#114)), sum(UnscaledValue(ss_list_price#115)), sum(UnscaledValue(ss_coupon_amt#116))] -Aggregate Attributes [4]: [count(1)#85, sum(UnscaledValue(ss_wholesale_cost#114))#86, sum(UnscaledValue(ss_list_price#115))#87, sum(UnscaledValue(ss_coupon_amt#116))#88] -Results [8]: [i_item_sk#166 AS item_sk#174, s_store_name#133 AS store_name#175, s_zip#134 AS store_zip#176, d_year#131 AS syear#177, count(1)#85 AS cnt#178, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#114))#86,17,2) AS s1#179, MakeDecimal(sum(UnscaledValue(ss_list_price#115))#87,17,2) AS s2#180, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#116))#88,17,2) AS s3#181] - -(176) Exchange -Input [8]: [item_sk#174, store_name#175, store_zip#176, syear#177, cnt#178, s1#179, s2#180, s3#181] -Arguments: hashpartitioning(item_sk#174, store_name#175, store_zip#176, 5), ENSURE_REQUIREMENTS, [plan_id=16] - -(177) Sort [codegen id : 50] -Input [8]: [item_sk#174, store_name#175, store_zip#176, syear#177, cnt#178, s1#179, s2#180, s3#181] -Arguments: [item_sk#174 ASC NULLS FIRST, store_name#175 ASC NULLS FIRST, store_zip#176 ASC NULLS FIRST], false, 0 - -(178) SortMergeJoin [codegen id : 51] -Left keys [3]: [item_sk#90, store_name#91, store_zip#92] -Right keys [3]: [item_sk#174, store_name#175, store_zip#176] +(163) ReusedExchange [Reuses operator id: 94] +Output [2]: [i_item_sk#153, i_product_name#154] + +(164) CometBroadcastHashJoin +Left output [17]: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] +Right output [2]: [i_item_sk#153, i_product_name#154] +Arguments: [ss_item_sk#93], [i_item_sk#153], Inner, BuildRight + +(165) CometProject +Input [19]: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, i_item_sk#153, i_product_name#154] +Arguments: [ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, d_year#129, d_year#131, s_store_name#120, s_zip#121, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, i_item_sk#153, i_product_name#154], [ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, d_year#129, d_year#131, s_store_name#120, s_zip#121, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, i_item_sk#153, i_product_name#154] + +(166) CometHashAggregate +Input [18]: [ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, d_year#129, d_year#131, s_store_name#120, s_zip#121, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, i_item_sk#153, i_product_name#154] +Keys [15]: [i_product_name#154, i_item_sk#153, s_store_name#120, s_zip#121, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, d_year#118, d_year#129, d_year#131] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#101)), partial_sum(UnscaledValue(ss_list_price#102)), partial_sum(UnscaledValue(ss_coupon_amt#103))] + +(167) CometHashAggregate +Input [19]: [i_product_name#154, i_item_sk#153, s_store_name#120, s_zip#121, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, d_year#118, d_year#129, d_year#131, count#72, sum#155, sum#156, sum#157] +Keys [15]: [i_product_name#154, i_item_sk#153, s_store_name#120, s_zip#121, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, d_year#118, d_year#129, d_year#131] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#101)), sum(UnscaledValue(ss_list_price#102)), sum(UnscaledValue(ss_coupon_amt#103))] + +(168) CometColumnarExchange +Input [8]: [item_sk#158, store_name#159, store_zip#160, syear#161, cnt#162, s1#163, s2#164, s3#165] +Arguments: hashpartitioning(item_sk#158, store_name#159, store_zip#160, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(169) CometSort +Input [8]: [item_sk#158, store_name#159, store_zip#160, syear#161, cnt#162, s1#163, s2#164, s3#165] +Arguments: [item_sk#158, store_name#159, store_zip#160, syear#161, cnt#162, s1#163, s2#164, s3#165], [item_sk#158 ASC NULLS FIRST, store_name#159 ASC NULLS FIRST, store_zip#160 ASC NULLS FIRST] + +(170) ColumnarToRow [codegen id : 2] +Input [8]: [item_sk#158, store_name#159, store_zip#160, syear#161, cnt#162, s1#163, s2#164, s3#165] + +(171) SortMergeJoin [codegen id : 3] +Left keys [3]: [item_sk#77, store_name#78, store_zip#79] +Right keys [3]: [item_sk#158, store_name#159, store_zip#160] Join type: Inner -Join condition: (cnt#178 <= cnt#102) +Join condition: (cnt#162 <= cnt#89) -(179) Project [codegen id : 51] -Output [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#179, s2#180, s3#181, syear#177, cnt#178] -Input [25]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, item_sk#174, store_name#175, store_zip#176, syear#177, cnt#178, s1#179, s2#180, s3#181] +(172) Project [codegen id : 3] +Output [21]: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162] +Input [25]: [product_name#76, item_sk#77, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, item_sk#158, store_name#159, store_zip#160, syear#161, cnt#162, s1#163, s2#164, s3#165] -(180) Exchange -Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#179, s2#180, s3#181, syear#177, cnt#178] -Arguments: rangepartitioning(product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#178 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=17] +(173) RowToColumnar +Input [21]: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162] -(181) Sort [codegen id : 52] -Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#179, s2#180, s3#181, syear#177, cnt#178] -Arguments: [product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#178 ASC NULLS FIRST], true, 0 +(174) CometColumnarExchange +Input [21]: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162] +Arguments: rangepartitioning(product_name#76 ASC NULLS FIRST, store_name#78 ASC NULLS FIRST, cnt#162 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(175) CometSort +Input [21]: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162] +Arguments: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162], [product_name#76 ASC NULLS FIRST, store_name#78 ASC NULLS FIRST, cnt#162 ASC NULLS FIRST] + +(176) ColumnarToRow [codegen id : 4] +Input [21]: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (185) -+- * ColumnarToRow (184) - +- CometFilter (183) - +- CometScan parquet spark_catalog.default.date_dim (182) +BroadcastExchange (180) ++- * ColumnarToRow (179) + +- CometFilter (178) + +- CometScan parquet spark_catalog.default.date_dim (177) -(182) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#37, d_year#38] +(177) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#32, d_year#33] 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 -(183) CometFilter -Input [2]: [d_date_sk#37, d_year#38] -Condition : ((isnotnull(d_year#38) AND (d_year#38 = 1999)) AND isnotnull(d_date_sk#37)) +(178) CometFilter +Input [2]: [d_date_sk#32, d_year#33] +Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(184) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#37, d_year#38] +(179) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#32, d_year#33] -(185) BroadcastExchange -Input [2]: [d_date_sk#37, d_year#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] +(180) BroadcastExchange +Input [2]: [d_date_sk#32, d_year#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -Subquery:2 Hosting operator id = 111 Hosting Expression = ss_sold_date_sk#117 IN dynamicpruning#118 -BroadcastExchange (189) -+- * ColumnarToRow (188) - +- CometFilter (187) - +- CometScan parquet spark_catalog.default.date_dim (186) +Subquery:2 Hosting operator id = 102 Hosting Expression = ss_sold_date_sk#104 IN dynamicpruning#105 +BroadcastExchange (184) ++- * ColumnarToRow (183) + +- CometFilter (182) + +- CometScan parquet spark_catalog.default.date_dim (181) -(186) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#130, d_year#131] +(181) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#117, d_year#118] 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 -(187) CometFilter -Input [2]: [d_date_sk#130, d_year#131] -Condition : ((isnotnull(d_year#131) AND (d_year#131 = 2000)) AND isnotnull(d_date_sk#130)) +(182) CometFilter +Input [2]: [d_date_sk#117, d_year#118] +Condition : ((isnotnull(d_year#118) AND (d_year#118 = 2000)) AND isnotnull(d_date_sk#117)) -(188) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#130, d_year#131] +(183) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#117, d_year#118] -(189) BroadcastExchange -Input [2]: [d_date_sk#130, d_year#131] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=19] +(184) BroadcastExchange +Input [2]: [d_date_sk#117, d_year#118] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/simplified.txt index 3b5e4f19e..84be0e3c3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/simplified.txt @@ -1,281 +1,199 @@ -WholeStageCodegen (52) - Sort [product_name,store_name,cnt] +WholeStageCodegen (4) + ColumnarToRow InputAdapter - Exchange [product_name,store_name,cnt] #1 - WholeStageCodegen (51) - Project [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - SortMergeJoin [item_sk,store_name,store_zip,item_sk,store_name,store_zip,cnt,cnt] - InputAdapter - WholeStageCodegen (25) - Sort [item_sk,store_name,store_zip] - InputAdapter - Exchange [item_sk,store_name,store_zip] #2 - WholeStageCodegen (24) - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] - Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SortMergeJoin [ss_item_sk,cs_item_sk] - InputAdapter - WholeStageCodegen (2) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #4 - CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - WholeStageCodegen (8) - Sort [cs_item_sk] - Project [cs_item_sk] - Filter [sale,refund] - HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] - InputAdapter - Exchange [cs_item_sk] #6 - WholeStageCodegen (7) - HashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [sum,sum,isEmpty,sum,sum,isEmpty] - Project [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] - InputAdapter - WholeStageCodegen (4) - Sort [cs_item_sk,cs_order_number] - InputAdapter - Exchange [cs_item_sk,cs_order_number] #7 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - InputAdapter - WholeStageCodegen (6) - Sort [cr_item_sk,cr_order_number] - InputAdapter - Exchange [cr_item_sk,cr_order_number] #8 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_name,s_zip] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (12) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #11 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (14) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk,cd_marital_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometFilter [p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk] - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (17) - 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 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (19) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 - InputAdapter - BroadcastExchange #16 - WholeStageCodegen (21) - ColumnarToRow - InputAdapter - CometFilter [ib_income_band_sk] - CometScan parquet spark_catalog.default.income_band [ib_income_band_sk] - InputAdapter - ReusedExchange [ib_income_band_sk] #16 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (23) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] - InputAdapter - WholeStageCodegen (50) - Sort [item_sk,store_name,store_zip] - InputAdapter - Exchange [item_sk,store_name,store_zip] #18 - WholeStageCodegen (49) - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] - Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SortMergeJoin [ss_item_sk,cs_item_sk] - InputAdapter - WholeStageCodegen (27) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #19 - WholeStageCodegen (26) - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #20 - CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #21 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - WholeStageCodegen (33) - Sort [cs_item_sk] - Project [cs_item_sk] - Filter [sale,refund] - HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] - InputAdapter - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #21 - InputAdapter - ReusedExchange [s_store_sk,s_store_name,s_zip] #9 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #10 - InputAdapter - ReusedExchange [d_date_sk,d_year] #11 - InputAdapter - ReusedExchange [d_date_sk,d_year] #11 - InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 - InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 - InputAdapter - ReusedExchange [p_promo_sk] #13 - InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 - InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 - InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 - InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 - InputAdapter - ReusedExchange [ib_income_band_sk] #16 - InputAdapter - ReusedExchange [ib_income_band_sk] #16 - InputAdapter - ReusedExchange [i_item_sk,i_product_name] #17 + CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] + CometColumnarExchange [product_name,store_name,cnt] #1 + RowToColumnar + WholeStageCodegen (3) + Project [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] + SortMergeJoin [item_sk,store_name,store_zip,item_sk,store_name,store_zip,cnt,cnt] + InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometSort [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3] + CometColumnarExchange [item_sk,store_name,store_zip] #2 + CometHashAggregate [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] + CometHashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum,ss_wholesale_cost,ss_list_price,ss_coupon_amt] + CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] + CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk] #3 + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #4 + CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [cs_item_sk,sale,refund] + CometHashAggregate [cs_item_sk,sale,refund,sum,sum,isEmpty,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] + CometColumnarExchange [cs_item_sk] #6 + CometHashAggregate [cs_item_sk,sum,sum,isEmpty,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometSortMergeJoin [cs_item_sk,cs_order_number,cs_ext_list_price,cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometSort [cs_item_sk,cs_order_number,cs_ext_list_price] + CometColumnarExchange [cs_item_sk,cs_order_number] #7 + CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometColumnarExchange [cr_item_sk,cr_order_number] #8 + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 + CometFilter [s_store_sk,s_store_name,s_zip] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #12 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk,d_year] #12 + CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 + CometFilter [cd_demo_sk,cd_marital_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + ReusedExchange [cd_demo_sk,cd_marital_status] #13 + CometBroadcastExchange [p_promo_sk] #14 + CometFilter [p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk] + CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 + CometFilter [hd_demo_sk,hd_income_band_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] + ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 + CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + CometBroadcastExchange [ib_income_band_sk] #17 + CometFilter [ib_income_band_sk] + CometScan parquet spark_catalog.default.income_band [ib_income_band_sk] + ReusedExchange [ib_income_band_sk] #17 + CometBroadcastExchange [i_item_sk,i_product_name] #18 + CometProject [i_item_sk,i_product_name] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] + InputAdapter + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometSort [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] + CometColumnarExchange [item_sk,store_name,store_zip] #19 + CometHashAggregate [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] + CometHashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum,ss_wholesale_cost,ss_list_price,ss_coupon_amt] + CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] + CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk] #20 + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #21 + CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #22 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [cs_item_sk,sale,refund] + CometHashAggregate [cs_item_sk,sale,refund,sum,sum,isEmpty,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] + ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 + CometBroadcastExchange [d_date_sk,d_year] #23 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [s_store_sk,s_store_name,s_zip] #10 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 + ReusedExchange [d_date_sk,d_year] #12 + ReusedExchange [d_date_sk,d_year] #12 + ReusedExchange [cd_demo_sk,cd_marital_status] #13 + ReusedExchange [cd_demo_sk,cd_marital_status] #13 + ReusedExchange [p_promo_sk] #14 + ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 + ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + ReusedExchange [ib_income_band_sk] #17 + ReusedExchange [ib_income_band_sk] #17 + ReusedExchange [i_item_sk,i_product_name] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/explain.txt index a10b6897c..dd1796dd4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/explain.txt @@ -1,46 +1,43 @@ == Physical Plan == -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 (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) +* ColumnarToRow (39) ++- CometTakeOrderedAndProject (38) + +- CometProject (37) + +- CometBroadcastHashJoin (36) + :- CometProject (22) + : +- CometBroadcastHashJoin (21) + : :- CometProject (17) + : : +- CometBroadcastHashJoin (16) + : : :- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store (1) + : : +- CometBroadcastExchange (15) + : : +- CometFilter (14) + : : +- CometHashAggregate (13) + : : +- CometColumnarExchange (12) + : : +- CometHashAggregate (11) + : : +- 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 (20) + : +- CometFilter (19) + : +- CometScan parquet spark_catalog.default.item (18) + +- CometBroadcastExchange (35) + +- CometFilter (34) + +- CometHashAggregate (33) + +- CometColumnarExchange (32) + +- CometHashAggregate (31) + +- CometHashAggregate (30) + +- CometColumnarExchange (29) + +- CometHashAggregate (28) + +- CometProject (27) + +- CometBroadcastHashJoin (26) + :- CometFilter (24) + : +- CometScan parquet spark_catalog.default.store_sales (23) + +- ReusedExchange (25) (1) Scan parquet spark_catalog.default.store @@ -54,10 +51,7 @@ ReadSchema: struct Input [2]: [s_store_sk#1, s_store_name#2] Condition : isnotnull(s_store_sk#1) -(3) ColumnarToRow [codegen id : 7] -Input [2]: [s_store_sk#1, s_store_name#2] - -(4) Scan parquet spark_catalog.default.store_sales +(3) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] Batched: true Location: InMemoryFileIndex [] @@ -65,218 +59,201 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter 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) Scan parquet spark_catalog.default.date_dim +(5) 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) CometFilter +(6) 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) CometProject +(7) CometProject Input [2]: [d_date_sk#8, d_month_seq#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(9) CometBroadcastExchange +(8) CometBroadcastExchange Input [1]: [d_date_sk#8] Arguments: [d_date_sk#8] -(10) CometBroadcastHashJoin +(9) 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 +(10) 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) CometHashAggregate +(11) 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))] -(13) ColumnarToRow [codegen id : 1] -Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] - -(14) Exchange +(12) CometColumnarExchange 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] +Arguments: hashpartitioning(ss_store_sk#4, ss_item_sk#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(15) HashAggregate [codegen id : 2] +(13) CometHashAggregate 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] -(16) Filter [codegen id : 2] -Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#12] -Condition : isnotnull(revenue#12) +(14) CometFilter +Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] +Condition : isnotnull(revenue#11) -(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) CometBroadcastExchange +Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] +Arguments: [ss_store_sk#4, ss_item_sk#3, revenue#11] -(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) CometBroadcastHashJoin +Left output [2]: [s_store_sk#1, s_store_name#2] +Right output [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] +Arguments: [s_store_sk#1], [ss_store_sk#4], Inner, BuildRight -(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) CometProject +Input [5]: [s_store_sk#1, s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11] +Arguments: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11], [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11] -(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] +(18) Scan parquet spark_catalog.default.item +Output [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(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) CometFilter +Input [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] +Condition : isnotnull(i_item_sk#12) -(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) CometBroadcastExchange +Input [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] +Arguments: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] -(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) CometBroadcastHashJoin +Left output [4]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11] +Right output [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] +Arguments: [ss_item_sk#3], [i_item_sk#12], Inner, BuildRight -(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) CometProject +Input [9]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11, i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] +Arguments: [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16], [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] -(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] - -(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] +(23) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_item_sk#17, ss_store_sk#18, ss_sales_price#19, ss_sold_date_sk#20] 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#20), dynamicpruningexpression(ss_sold_date_sk#20 IN dynamicpruning#21)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(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) +(24) CometFilter +Input [4]: [ss_item_sk#17, ss_store_sk#18, ss_sales_price#19, ss_sold_date_sk#20] +Condition : isnotnull(ss_store_sk#18) + +(25) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#22] + +(26) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#17, ss_store_sk#18, ss_sales_price#19, ss_sold_date_sk#20] +Right output [1]: [d_date_sk#22] +Arguments: [ss_sold_date_sk#20], [d_date_sk#22], Inner, BuildRight + +(27) CometProject +Input [5]: [ss_item_sk#17, ss_store_sk#18, ss_sales_price#19, ss_sold_date_sk#20, d_date_sk#22] +Arguments: [ss_item_sk#17, ss_store_sk#18, ss_sales_price#19], [ss_item_sk#17, ss_store_sk#18, ss_sales_price#19] -(28) ReusedExchange [Reuses operator id: 9] -Output [1]: [d_date_sk#23] +(28) CometHashAggregate +Input [3]: [ss_item_sk#17, ss_store_sk#18, ss_sales_price#19] +Keys [2]: [ss_store_sk#18, ss_item_sk#17] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#19))] -(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 +(29) CometColumnarExchange +Input [3]: [ss_store_sk#18, ss_item_sk#17, sum#23] +Arguments: hashpartitioning(ss_store_sk#18, ss_item_sk#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(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] +(30) CometHashAggregate +Input [3]: [ss_store_sk#18, ss_item_sk#17, sum#23] +Keys [2]: [ss_store_sk#18, ss_item_sk#17] +Functions [1]: [sum(UnscaledValue(ss_sales_price#19))] (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))] - -(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] - -(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))#25] -Results [2]: [ss_store_sk#19, MakeDecimal(sum(UnscaledValue(ss_sales_price#20))#25,17,2) AS revenue#26] - -(35) HashAggregate [codegen id : 5] -Input [2]: [ss_store_sk#19, revenue#26] -Keys [1]: [ss_store_sk#19] -Functions [1]: [partial_avg(revenue#26)] -Aggregate Attributes [2]: [sum#27, count#28] -Results [3]: [ss_store_sk#19, sum#29, count#30] - -(36) Exchange -Input [3]: [ss_store_sk#19, sum#29, count#30] -Arguments: hashpartitioning(ss_store_sk#19, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(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#26)] -Aggregate Attributes [1]: [avg(revenue#26)#31] -Results [2]: [ss_store_sk#19, avg(revenue#26)#31 AS ave#32] - -(38) Filter [codegen id : 6] -Input [2]: [ss_store_sk#19, ave#32] -Condition : isnotnull(ave#32) - -(39) BroadcastExchange -Input [2]: [ss_store_sk#19, ave#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -(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#32)) - -(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#32] - -(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] +Input [2]: [ss_store_sk#18, revenue#24] +Keys [1]: [ss_store_sk#18] +Functions [1]: [partial_avg(revenue#24)] + +(32) CometColumnarExchange +Input [3]: [ss_store_sk#18, sum#25, count#26] +Arguments: hashpartitioning(ss_store_sk#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(33) CometHashAggregate +Input [3]: [ss_store_sk#18, sum#25, count#26] +Keys [1]: [ss_store_sk#18] +Functions [1]: [avg(revenue#24)] + +(34) CometFilter +Input [2]: [ss_store_sk#18, ave#27] +Condition : isnotnull(ave#27) + +(35) CometBroadcastExchange +Input [2]: [ss_store_sk#18, ave#27] +Arguments: [ss_store_sk#18, ave#27] + +(36) CometBroadcastHashJoin +Left output [7]: [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] +Right output [2]: [ss_store_sk#18, ave#27] +Arguments: [ss_store_sk#4], [ss_store_sk#18], Inner, (cast(revenue#11 as decimal(23,7)) <= (0.1 * ave#27)), BuildRight + +(37) CometProject +Input [9]: [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16, ss_store_sk#18, ave#27] +Arguments: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#16], [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#16] + +(38) CometTakeOrderedAndProject +Input [6]: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#16] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#2 ASC NULLS FIRST,i_item_desc#13 ASC NULLS FIRST], output=[s_store_name#2,i_item_desc#13,revenue#11,i_current_price#14,i_wholesale_cost#15,i_brand#16]), [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#16], 100, [s_store_name#2 ASC NULLS FIRST, i_item_desc#13 ASC NULLS FIRST], [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#16] + +(39) ColumnarToRow [codegen id : 1] +Input [6]: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#16] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -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#6 IN dynamicpruning#7 +BroadcastExchange (44) ++- * ColumnarToRow (43) + +- CometProject (42) + +- CometFilter (41) + +- CometScan parquet spark_catalog.default.date_dim (40) -(43) Scan parquet spark_catalog.default.date_dim +(40) 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 -(44) CometFilter +(41) 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)) -(45) CometProject +(42) CometProject Input [2]: [d_date_sk#8, d_month_seq#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(46) ColumnarToRow [codegen id : 1] +(43) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#8] -(47) BroadcastExchange +(44) BroadcastExchange Input [1]: [d_date_sk#8] -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=4] -Subquery:2 Hosting operator id = 26 Hosting Expression = ss_sold_date_sk#21 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/simplified.txt index 6cb247973..e41f9161d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/simplified.txt @@ -1,68 +1,50 @@ -TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - 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] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [s_store_name,ss_store_sk,ss_item_sk,revenue] - BroadcastHashJoin [s_store_sk,ss_store_sk] - ColumnarToRow - InputAdapter +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] + CometProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] + CometBroadcastHashJoin [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand,ss_store_sk,ave] + CometProject [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand] + CometBroadcastHashJoin [s_store_name,ss_store_sk,ss_item_sk,revenue,i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] + CometProject [s_store_name,ss_store_sk,ss_item_sk,revenue] + CometBroadcastHashJoin [s_store_sk,s_store_name,ss_store_sk,ss_item_sk,revenue] CometFilter [s_store_sk,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] - InputAdapter - BroadcastExchange #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ss_store_sk,ss_item_sk,sum,ss_sales_price] - CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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 #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - Filter [ave] - HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count] - InputAdapter - 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] #8 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometHashAggregate [ss_store_sk,ss_item_sk,sum,ss_sales_price] - CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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 - ReusedExchange [d_date_sk] #4 + CometBroadcastExchange [ss_store_sk,ss_item_sk,revenue] #1 + CometFilter [ss_store_sk,ss_item_sk,revenue] + CometHashAggregate [ss_store_sk,ss_item_sk,revenue,sum,sum(UnscaledValue(ss_sales_price))] + CometColumnarExchange [ss_store_sk,ss_item_sk] #2 + CometHashAggregate [ss_store_sk,ss_item_sk,sum,ss_sales_price] + CometProject [ss_item_sk,ss_store_sk,ss_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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 #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] #5 + CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] + CometBroadcastExchange [ss_store_sk,ave] #6 + CometFilter [ss_store_sk,ave] + CometHashAggregate [ss_store_sk,ave,sum,count,avg(revenue)] + CometColumnarExchange [ss_store_sk] #7 + CometHashAggregate [ss_store_sk,sum,count,revenue] + CometHashAggregate [ss_store_sk,revenue,ss_item_sk,sum,sum(UnscaledValue(ss_sales_price))] + CometColumnarExchange [ss_store_sk,ss_item_sk] #8 + CometHashAggregate [ss_store_sk,ss_item_sk,sum,ss_sales_price] + CometProject [ss_item_sk,ss_store_sk,ss_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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 + ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/explain.txt index 6bca75e27..d0c1814d1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/explain.txt @@ -1,55 +1,54 @@ == Physical Plan == -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) +* ColumnarToRow (50) ++- CometTakeOrderedAndProject (49) + +- CometHashAggregate (48) + +- CometColumnarExchange (47) + +- CometHashAggregate (46) + +- CometUnion (45) + :- CometHashAggregate (27) + : +- CometColumnarExchange (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) + +- CometHashAggregate (44) + +- CometColumnarExchange (43) + +- CometHashAggregate (42) + +- CometProject (41) + +- CometBroadcastHashJoin (40) + :- CometProject (38) + : +- CometBroadcastHashJoin (37) + : :- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometProject (32) + : : : +- CometBroadcastHashJoin (31) + : : : :- CometFilter (29) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (28) + : : : +- ReusedExchange (30) + : : +- ReusedExchange (33) + : +- ReusedExchange (36) + +- ReusedExchange (39) (1) Scan parquet spark_catalog.default.web_sales @@ -173,150 +172,141 @@ Input [11]: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_nam 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)] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(28) HashAggregate [codegen id : 2] +(27) CometHashAggregate 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)#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] -(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] +(28) Scan parquet spark_catalog.default.catalog_sales +Output [7]: [cs_sold_time_sk#71, cs_ship_mode_sk#72, cs_warehouse_sk#73, cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, cs_sold_date_sk#77] 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#77), dynamicpruningexpression(cs_sold_date_sk#77 IN dynamicpruning#78)] PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_sold_time_sk), IsNotNull(cs_ship_mode_sk)] ReadSchema: struct -(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] +(29) CometFilter +Input [7]: [cs_sold_time_sk#71, cs_ship_mode_sk#72, cs_warehouse_sk#73, cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, cs_sold_date_sk#77] +Condition : ((isnotnull(cs_warehouse_sk#73) AND isnotnull(cs_sold_time_sk#71)) AND isnotnull(cs_ship_mode_sk#72)) + +(30) ReusedExchange [Reuses operator id: 5] +Output [7]: [w_warehouse_sk#79, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85] + +(31) CometBroadcastHashJoin +Left output [7]: [cs_sold_time_sk#71, cs_ship_mode_sk#72, cs_warehouse_sk#73, cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, cs_sold_date_sk#77] +Right output [7]: [w_warehouse_sk#79, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85] +Arguments: [cs_warehouse_sk#73], [w_warehouse_sk#79], Inner, BuildRight + +(32) CometProject +Input [14]: [cs_sold_time_sk#71, cs_ship_mode_sk#72, cs_warehouse_sk#73, cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, cs_sold_date_sk#77, w_warehouse_sk#79, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85] +Arguments: [cs_sold_time_sk#71, cs_ship_mode_sk#72, cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, cs_sold_date_sk#77, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85], [cs_sold_time_sk#71, cs_ship_mode_sk#72, cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, cs_sold_date_sk#77, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85] + +(33) ReusedExchange [Reuses operator id: 10] +Output [3]: [d_date_sk#86, d_year#87, d_moy#88] + +(34) CometBroadcastHashJoin +Left output [12]: [cs_sold_time_sk#71, cs_ship_mode_sk#72, cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, cs_sold_date_sk#77, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85] +Right output [3]: [d_date_sk#86, d_year#87, d_moy#88] +Arguments: [cs_sold_date_sk#77], [d_date_sk#86], Inner, BuildRight + +(35) CometProject +Input [15]: [cs_sold_time_sk#71, cs_ship_mode_sk#72, cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, cs_sold_date_sk#77, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, d_date_sk#86, d_year#87, d_moy#88] +Arguments: [cs_sold_time_sk#71, cs_ship_mode_sk#72, cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, d_year#87, d_moy#88], [cs_sold_time_sk#71, cs_ship_mode_sk#72, cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, d_year#87, d_moy#88] + +(36) ReusedExchange [Reuses operator id: 16] +Output [1]: [t_time_sk#89] + +(37) CometBroadcastHashJoin +Left output [13]: [cs_sold_time_sk#71, cs_ship_mode_sk#72, cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, d_year#87, d_moy#88] +Right output [1]: [t_time_sk#89] +Arguments: [cs_sold_time_sk#71], [t_time_sk#89], Inner, BuildRight + +(38) CometProject +Input [14]: [cs_sold_time_sk#71, cs_ship_mode_sk#72, cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, d_year#87, d_moy#88, t_time_sk#89] +Arguments: [cs_ship_mode_sk#72, cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, d_year#87, d_moy#88], [cs_ship_mode_sk#72, cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, d_year#87, d_moy#88] + +(39) ReusedExchange [Reuses operator id: 22] +Output [1]: [sm_ship_mode_sk#90] + +(40) CometBroadcastHashJoin +Left output [12]: [cs_ship_mode_sk#72, cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, d_year#87, d_moy#88] +Right output [1]: [sm_ship_mode_sk#90] +Arguments: [cs_ship_mode_sk#72], [sm_ship_mode_sk#90], Inner, BuildRight + +(41) CometProject +Input [13]: [cs_ship_mode_sk#72, cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, d_year#87, d_moy#88, sm_ship_mode_sk#90] +Arguments: [cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, d_year#87, d_moy#88], [cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, d_year#87, d_moy#88] + +(42) CometHashAggregate +Input [11]: [cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, d_year#87, d_moy#88] +Keys [7]: [w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, d_year#87] +Functions [24]: [partial_sum(CASE WHEN (d_moy#88 = 1) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 2) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 3) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 4) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 5) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 6) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 7) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 8) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 9) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 10) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 11) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 12) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 1) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 2) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 3) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 4) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 5) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 6) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 7) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 8) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 9) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 10) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 11) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 12) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END)] + +(43) CometColumnarExchange +Input [55]: [w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, d_year#87, 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, sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] +Arguments: hashpartitioning(w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, d_year#87, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(44) CometHashAggregate +Input [55]: [w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, d_year#87, 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, sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] +Keys [7]: [w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, d_year#87] +Functions [24]: [sum(CASE WHEN (d_moy#88 = 1) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 2) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 3) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 4) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 5) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 6) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 7) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 8) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 9) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 10) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 11) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 12) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 1) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 2) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 3) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 4) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 5) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 6) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 7) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 8) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 9) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 10) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 11) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 12) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END)] + +(45) CometUnion +Child 0 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#139, year#140, jan_sales#141, feb_sales#142, mar_sales#143, apr_sales#144, may_sales#145, jun_sales#146, jul_sales#147, aug_sales#148, sep_sales#149, oct_sales#150, nov_sales#151, dec_sales#152, jan_net#153, feb_net#154, mar_net#155, apr_net#156, may_net#157, jun_net#158, jul_net#159, aug_net#160, sep_net#161, oct_net#162, nov_net#163, dec_net#164] +Child 1 Input [32]: [w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, ship_carriers#165, year#166, jan_sales#167, feb_sales#168, mar_sales#169, apr_sales#170, may_sales#171, jun_sales#172, jul_sales#173, aug_sales#174, sep_sales#175, oct_sales#176, nov_sales#177, dec_sales#178, jan_net#179, feb_net#180, mar_net#181, apr_net#182, may_net#183, jun_net#184, jul_net#185, aug_net#186, sep_net#187, oct_net#188, nov_net#189, dec_net#190] + +(46) CometHashAggregate +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#139, year#140, jan_sales#141, feb_sales#142, mar_sales#143, apr_sales#144, may_sales#145, jun_sales#146, jul_sales#147, aug_sales#148, sep_sales#149, oct_sales#150, nov_sales#151, dec_sales#152, jan_net#153, feb_net#154, mar_net#155, apr_net#156, may_net#157, jun_net#158, jul_net#159, aug_net#160, sep_net#161, oct_net#162, nov_net#163, dec_net#164] +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#139, year#140] +Functions [36]: [partial_sum(jan_sales#141), partial_sum(feb_sales#142), partial_sum(mar_sales#143), partial_sum(apr_sales#144), partial_sum(may_sales#145), partial_sum(jun_sales#146), partial_sum(jul_sales#147), partial_sum(aug_sales#148), partial_sum(sep_sales#149), partial_sum(oct_sales#150), partial_sum(nov_sales#151), partial_sum(dec_sales#152), partial_sum((jan_sales#141 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((feb_sales#142 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((mar_sales#143 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((apr_sales#144 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((may_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jun_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jul_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((aug_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((sep_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((oct_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((nov_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((dec_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum(jan_net#153), partial_sum(feb_net#154), partial_sum(mar_net#155), partial_sum(apr_net#156), partial_sum(may_net#157), partial_sum(jun_net#158), partial_sum(jul_net#159), partial_sum(aug_net#160), partial_sum(sep_net#161), partial_sum(oct_net#162), partial_sum(nov_net#163), partial_sum(dec_net#164)] + +(47) CometColumnarExchange +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#139, year#140, 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, 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] +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#139, year#140, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(48) CometHashAggregate +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#139, year#140, 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, 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] +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#139, year#140] +Functions [36]: [sum(jan_sales#141), sum(feb_sales#142), sum(mar_sales#143), sum(apr_sales#144), sum(may_sales#145), sum(jun_sales#146), sum(jul_sales#147), sum(aug_sales#148), sum(sep_sales#149), sum(oct_sales#150), sum(nov_sales#151), sum(dec_sales#152), sum((jan_sales#141 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((feb_sales#142 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((mar_sales#143 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((apr_sales#144 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((may_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jun_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jul_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((aug_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((sep_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((oct_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((nov_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((dec_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum(jan_net#153), sum(feb_net#154), sum(mar_net#155), sum(apr_net#156), sum(may_net#157), sum(jun_net#158), sum(jul_net#159), sum(aug_net#160), sum(sep_net#161), sum(oct_net#162), sum(nov_net#163), sum(dec_net#164)] + +(49) CometTakeOrderedAndProject +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#139, year#140, jan_sales#263, feb_sales#264, mar_sales#265, apr_sales#266, may_sales#267, jun_sales#268, jul_sales#269, aug_sales#270, sep_sales#271, oct_sales#272, nov_sales#273, dec_sales#274, jan_sales_per_sq_foot#275, feb_sales_per_sq_foot#276, mar_sales_per_sq_foot#277, apr_sales_per_sq_foot#278, may_sales_per_sq_foot#279, jun_sales_per_sq_foot#280, jul_sales_per_sq_foot#281, aug_sales_per_sq_foot#282, sep_sales_per_sq_foot#283, oct_sales_per_sq_foot#284, nov_sales_per_sq_foot#285, dec_sales_per_sq_foot#286, jan_net#287, feb_net#288, mar_net#289, apr_net#290, may_net#291, jun_net#292, jul_net#293, aug_net#294, sep_net#295, oct_net#296, nov_net#297, dec_net#298] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_warehouse_name#10 ASC NULLS FIRST], output=[w_warehouse_name#10,w_warehouse_sq_ft#11,w_city#12,w_county#13,w_state#14,w_country#15,ship_carriers#139,year#140,jan_sales#263,feb_sales#264,mar_sales#265,apr_sales#266,may_sales#267,jun_sales#268,jul_sales#269,aug_sales#270,sep_sales#271,oct_sales#272,nov_sales#273,dec_sales#274,jan_sales_per_sq_foot#275,feb_sales_per_sq_foot#276,mar_sales_per_sq_foot#277,apr_sales_per_sq_foot#278,may_sales_per_sq_foot#279,jun_sales_per_sq_foot#280,jul_sales_per_sq_foot#281,aug_sales_per_sq_foot#282,sep_sales_per_sq_foot#283,oct_sales_per_sq_foot#284,nov_sales_per_sq_foot#285,dec_sales_per_sq_foot#286,jan_net#287,feb_net#288,mar_net#289,apr_net#290,may_net#291,jun_net#292,jul_net#293,aug_net#294,sep_net#295,oct_net#296,nov_net#297,dec_net#298]), [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#139, year#140, jan_sales#263, feb_sales#264, mar_sales#265, apr_sales#266, may_sales#267, jun_sales#268, jul_sales#269, aug_sales#270, sep_sales#271, oct_sales#272, nov_sales#273, dec_sales#274, jan_sales_per_sq_foot#275, feb_sales_per_sq_foot#276, mar_sales_per_sq_foot#277, apr_sales_per_sq_foot#278, may_sales_per_sq_foot#279, jun_sales_per_sq_foot#280, jul_sales_per_sq_foot#281, aug_sales_per_sq_foot#282, sep_sales_per_sq_foot#283, oct_sales_per_sq_foot#284, nov_sales_per_sq_foot#285, dec_sales_per_sq_foot#286, jan_net#287, feb_net#288, mar_net#289, apr_net#290, may_net#291, jun_net#292, jul_net#293, aug_net#294, sep_net#295, oct_net#296, nov_net#297, dec_net#298], 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#139, year#140, jan_sales#263, feb_sales#264, mar_sales#265, apr_sales#266, may_sales#267, jun_sales#268, jul_sales#269, aug_sales#270, sep_sales#271, oct_sales#272, nov_sales#273, dec_sales#274, jan_sales_per_sq_foot#275, feb_sales_per_sq_foot#276, mar_sales_per_sq_foot#277, apr_sales_per_sq_foot#278, may_sales_per_sq_foot#279, jun_sales_per_sq_foot#280, jul_sales_per_sq_foot#281, aug_sales_per_sq_foot#282, sep_sales_per_sq_foot#283, oct_sales_per_sq_foot#284, nov_sales_per_sq_foot#285, dec_sales_per_sq_foot#286, jan_net#287, feb_net#288, mar_net#289, apr_net#290, may_net#291, jun_net#292, jul_net#293, aug_net#294, sep_net#295, oct_net#296, nov_net#297, dec_net#298] + +(50) ColumnarToRow [codegen id : 1] +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#139, year#140, jan_sales#263, feb_sales#264, mar_sales#265, apr_sales#266, may_sales#267, jun_sales#268, jul_sales#269, aug_sales#270, sep_sales#271, oct_sales#272, nov_sales#273, dec_sales#274, jan_sales_per_sq_foot#275, feb_sales_per_sq_foot#276, mar_sales_per_sq_foot#277, apr_sales_per_sq_foot#278, may_sales_per_sq_foot#279, jun_sales_per_sq_foot#280, jul_sales_per_sq_foot#281, aug_sales_per_sq_foot#282, sep_sales_per_sq_foot#283, oct_sales_per_sq_foot#284, nov_sales_per_sq_foot#285, dec_sales_per_sq_foot#286, jan_net#287, feb_net#288, mar_net#289, apr_net#290, may_net#291, jun_net#292, jul_net#293, aug_net#294, sep_net#295, oct_net#296, nov_net#297, dec_net#298] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (55) -+- * ColumnarToRow (54) - +- CometFilter (53) - +- CometScan parquet spark_catalog.default.date_dim (52) +BroadcastExchange (54) ++- * ColumnarToRow (53) + +- CometFilter (52) + +- CometScan parquet spark_catalog.default.date_dim (51) -(52) Scan parquet spark_catalog.default.date_dim +(51) 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 -(53) CometFilter +(52) 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)) -(54) ColumnarToRow [codegen id : 1] +(53) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#16, d_year#17, d_moy#18] -(55) BroadcastExchange +(54) 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=4] -Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#127 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 28 Hosting Expression = cs_sold_date_sk#77 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/simplified.txt index 72133811b..ce4354bc5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/simplified.txt @@ -1,71 +1,60 @@ -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 (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 (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 (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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [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,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,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,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,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,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_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,d_date_sk,d_year,d_moy] - 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_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometFilter [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 - CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - 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 [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [t_time_sk] #6 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_time] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] - CometBroadcastExchange [sm_ship_mode_sk] #7 - CometProject [sm_ship_mode_sk] - CometFilter [sm_ship_mode_sk,sm_carrier] - 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] #8 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometHashAggregate [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,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,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,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,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,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_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,d_date_sk,d_year,d_moy] - 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_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometFilter [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] - 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 - 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 +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [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] + CometHashAggregate [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,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,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)] + CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year] #1 + CometHashAggregate [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,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] + CometUnion [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] + CometHashAggregate [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,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)] + CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #2 + CometHashAggregate [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,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,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,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,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,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_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,d_date_sk,d_year,d_moy] + 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_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometFilter [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 + CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + 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 [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [t_time_sk] #6 + CometProject [t_time_sk] + CometFilter [t_time_sk,t_time] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] + CometBroadcastExchange [sm_ship_mode_sk] #7 + CometProject [sm_ship_mode_sk] + CometFilter [sm_ship_mode_sk,sm_carrier] + CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] + CometHashAggregate [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,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)] + CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #8 + CometHashAggregate [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,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,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,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,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,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_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,d_date_sk,d_year,d_moy] + 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_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometFilter [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] + 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 + 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-spark3_5/q67/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/explain.txt index 9f26bde11..a9602820d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/explain.txt @@ -1,35 +1,37 @@ == Physical Plan == -TakeOrderedAndProject (31) -+- * Filter (30) - +- Window (29) - +- WindowGroupLimit (28) - +- * Sort (27) - +- Exchange (26) - +- WindowGroupLimit (25) - +- * Sort (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) +TakeOrderedAndProject (33) ++- * Filter (32) + +- Window (31) + +- WindowGroupLimit (30) + +- * ColumnarToRow (29) + +- CometSort (28) + +- CometColumnarExchange (27) + +- RowToColumnar (26) + +- WindowGroupLimit (25) + +- * ColumnarToRow (24) + +- CometSort (23) + +- CometHashAggregate (22) + +- CometColumnarExchange (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 @@ -129,81 +131,85 @@ Input [11]: [ss_quantity#3, ss_sales_price#4, i_category#19, i_class#20, i_brand 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))] -(21) ColumnarToRow [codegen id : 1] +(21) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(23) HashAggregate [codegen id : 2] +(22) CometHashAggregate 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))#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] -(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#31] -Arguments: [i_category#19 ASC NULLS FIRST, sumsales#31 DESC NULLS LAST], false, 0 +(23) CometSort +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#30] +Arguments: [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#30], [i_category#19 ASC NULLS FIRST, sumsales#30 DESC NULLS LAST] + +(24) ColumnarToRow [codegen id : 1] +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#30] (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#31] -Arguments: [i_category#19], [sumsales#31 DESC NULLS LAST], rank(sumsales#31), 100, Partial +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#30] +Arguments: [i_category#19], [sumsales#30 DESC NULLS LAST], rank(sumsales#30), 100, Partial + +(26) RowToColumnar +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#30] + +(27) CometColumnarExchange +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#30] +Arguments: hashpartitioning(i_category#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(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#31] -Arguments: hashpartitioning(i_category#19, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(28) CometSort +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#30] +Arguments: [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#30], [i_category#19 ASC NULLS FIRST, sumsales#30 DESC NULLS LAST] -(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#31] -Arguments: [i_category#19 ASC NULLS FIRST, sumsales#31 DESC NULLS LAST], false, 0 +(29) ColumnarToRow [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#30] -(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#31] -Arguments: [i_category#19], [sumsales#31 DESC NULLS LAST], rank(sumsales#31), 100, Final +(30) 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#30] +Arguments: [i_category#19], [sumsales#30 DESC NULLS LAST], rank(sumsales#30), 100, Final -(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#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] +(31) 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#30] +Arguments: [rank(sumsales#30) windowspecdefinition(i_category#19, sumsales#30 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#31], [i_category#19], [sumsales#30 DESC NULLS LAST] -(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#31, rk#32] -Condition : (rk#32 <= 100) +(32) Filter [codegen id : 3] +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#30, rk#31] +Condition : (rk#31 <= 100) -(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#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] +(33) 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#30, rk#31] +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#30 ASC NULLS FIRST, rk#31 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#30, rk#31] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (36) -+- * ColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan parquet spark_catalog.default.date_dim (32) +BroadcastExchange (38) ++- * ColumnarToRow (37) + +- CometProject (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.date_dim (34) -(32) Scan parquet spark_catalog.default.date_dim +(34) 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 -(33) CometFilter +(35) 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)) -(34) CometProject +(36) 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] -(35) ColumnarToRow [codegen id : 1] +(37) ColumnarToRow [codegen id : 1] Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(36) BroadcastExchange +(38) 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-spark3_5/q67/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/simplified.txt index 4dc4bcaf5..24aa5d08d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/simplified.txt @@ -1,22 +1,22 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (4) + WholeStageCodegen (3) Filter [rk] InputAdapter Window [sumsales,i_category] WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (3) - Sort [i_category,sumsales] + WholeStageCodegen (2) + ColumnarToRow InputAdapter - Exchange [i_category] #1 - WindowGroupLimit [i_category,sumsales] - 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 (1) - ColumnarToRow - InputAdapter + CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometColumnarExchange [i_category] #1 + RowToColumnar + WindowGroupLimit [i_category,sumsales] + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,spark_grouping_id,sum,isEmpty,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty,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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/explain.txt index 53443aedc..c07b9bcb9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/explain.txt @@ -1,46 +1,42 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * Project (41) - +- * BroadcastHashJoin Inner BuildRight (40) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * HashAggregate (29) - : : +- Exchange (28) - : : +- * 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) - +- BroadcastExchange (39) - +- * ColumnarToRow (38) - +- CometFilter (37) - +- CometScan parquet spark_catalog.default.customer_address (36) +* ColumnarToRow (38) ++- CometTakeOrderedAndProject (37) + +- CometProject (36) + +- CometBroadcastHashJoin (35) + :- CometProject (33) + : +- CometBroadcastHashJoin (32) + : :- CometHashAggregate (28) + : : +- CometColumnarExchange (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) + : +- CometBroadcastExchange (31) + : +- CometFilter (30) + : +- CometScan parquet spark_catalog.default.customer (29) + +- ReusedExchange (34) (1) Scan parquet spark_catalog.default.store_sales @@ -168,110 +164,88 @@ Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_pri 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))] -(27) ColumnarToRow [codegen id : 1] +(27) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(28) Exchange -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 : 4] +(28) CometHashAggregate 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))#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#31, c_current_addr_sk#32, c_first_name#33, c_last_name#34] +(29) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#24, c_current_addr_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), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(31) CometFilter -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 : 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#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 : 4] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#31] -Join type: Inner -Join condition: None +(30) CometFilter +Input [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#26, c_last_name#27] +Condition : (isnotnull(c_customer_sk#24) AND isnotnull(c_current_addr_sk#25)) -(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] +(31) CometBroadcastExchange +Input [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#26, c_last_name#27] +Arguments: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#26, c_last_name#27] -(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 +(32) CometBroadcastHashJoin +Left output [6]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#28, extended_price#29, list_price#30, extended_tax#31] +Right output [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#26, c_last_name#27] +Arguments: [ss_customer_sk#1], [c_customer_sk#24], Inner, BuildRight -(37) CometFilter -Input [2]: [ca_address_sk#35, ca_city#36] -Condition : (isnotnull(ca_address_sk#35) AND isnotnull(ca_city#36)) +(33) CometProject +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#24, c_current_addr_sk#25, c_first_name#26, c_last_name#27] +Arguments: [ss_ticket_number#5, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_current_addr_sk#25, c_first_name#26, c_last_name#27], [ss_ticket_number#5, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_current_addr_sk#25, c_first_name#26, c_last_name#27] -(38) ColumnarToRow [codegen id : 3] -Input [2]: [ca_address_sk#35, ca_city#36] +(34) ReusedExchange [Reuses operator id: 23] +Output [2]: [ca_address_sk#32, ca_city#33] -(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] +(35) CometBroadcastHashJoin +Left output [8]: [ss_ticket_number#5, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_current_addr_sk#25, c_first_name#26, c_last_name#27] +Right output [2]: [ca_address_sk#32, ca_city#33] +Arguments: [c_current_addr_sk#25], [ca_address_sk#32], Inner, NOT (ca_city#33 = bought_city#28), BuildRight -(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#36 = bought_city#27) +(36) CometProject +Input [10]: [ss_ticket_number#5, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_current_addr_sk#25, c_first_name#26, c_last_name#27, ca_address_sk#32, ca_city#33] +Arguments: [c_last_name#27, c_first_name#26, ca_city#33, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30], [c_last_name#27, c_first_name#26, ca_city#33, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] -(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] +(37) CometTakeOrderedAndProject +Input [8]: [c_last_name#27, c_first_name#26, ca_city#33, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#27 ASC NULLS FIRST,ss_ticket_number#5 ASC NULLS FIRST], output=[c_last_name#27,c_first_name#26,ca_city#33,bought_city#28,ss_ticket_number#5,extended_price#29,extended_tax#31,list_price#30]), [c_last_name#27, c_first_name#26, ca_city#33, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30], 100, [c_last_name#27 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#27, c_first_name#26, ca_city#33, 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] +(38) ColumnarToRow [codegen id : 1] +Input [8]: [c_last_name#27, c_first_name#26, ca_city#33, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (47) -+- * ColumnarToRow (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan parquet spark_catalog.default.date_dim (43) +BroadcastExchange (43) ++- * ColumnarToRow (42) + +- CometProject (41) + +- CometFilter (40) + +- CometScan parquet spark_catalog.default.date_dim (39) -(43) Scan parquet spark_catalog.default.date_dim +(39) 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 -(44) CometFilter +(40) 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)) -(45) CometProject +(41) CometProject Input [3]: [d_date_sk#11, d_year#12, d_dom#13] Arguments: [d_date_sk#11], [d_date_sk#11] -(46) ColumnarToRow [codegen id : 1] +(42) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(47) BroadcastExchange +(43) 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=2] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/simplified.txt index c5f56a69c..426f7c489 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/simplified.txt @@ -1,60 +1,48 @@ -TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_city,extended_price,extended_tax,list_price] - 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] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum,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_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_address_sk,ca_city] - CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,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_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,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_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,d_date_sk] - CometFilter [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] - 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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_city] - CometScan parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange [hd_demo_sk] #5 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [ca_address_sk,ca_city] #6 - CometFilter [ca_address_sk,ca_city] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_city] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] + CometProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] + CometBroadcastHashJoin [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk,ca_city] + CometProject [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name] + CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,bought_city,extended_price,list_price,extended_tax,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometHashAggregate [ss_ticket_number,ss_customer_sk,bought_city,extended_price,list_price,extended_tax,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))] + CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum,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_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_address_sk,ca_city] + CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,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_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,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_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,d_date_sk] + CometFilter [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] + 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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange [s_store_sk] #4 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_city] + CometScan parquet spark_catalog.default.store [s_store_sk,s_city] + CometBroadcastExchange [hd_demo_sk] #5 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange [ca_address_sk,ca_city] #6 + CometFilter [ca_address_sk,ca_city] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] #7 + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + ReusedExchange [ca_address_sk,ca_city] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/explain.txt index b931de292..4991d569f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/explain.txt @@ -1,48 +1,50 @@ == Physical Plan == -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) +TakeOrderedAndProject (46) ++- * HashAggregate (45) + +- * ColumnarToRow (44) + +- CometColumnarExchange (43) + +- RowToColumnar (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 @@ -238,50 +240,56 @@ Functions [1]: [partial_count(1)] 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] -(42) Exchange +(42) RowToColumnar 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] -(43) HashAggregate [codegen id : 6] +(43) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=5] + +(44) ColumnarToRow [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] + +(45) 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)#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] -(44) TakeOrderedAndProject +(46) 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 = 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) +BroadcastExchange (51) ++- * ColumnarToRow (50) + +- CometProject (49) + +- CometFilter (48) + +- CometScan parquet spark_catalog.default.date_dim (47) -(45) Scan parquet spark_catalog.default.date_dim +(47) 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 -(46) CometFilter +(48) 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)) -(47) CometProject +(49) CometProject Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(48) ColumnarToRow [codegen id : 1] +(50) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(49) BroadcastExchange +(51) BroadcastExchange Input [1]: [d_date_sk#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-v1_4-spark3_5/q69/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/simplified.txt index c9cc4959b..9e1e3edb6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/simplified.txt @@ -1,70 +1,72 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cnt1,cnt2,cnt3] 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 (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] - 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,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow + ColumnarToRow + InputAdapter + CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] #1 + RowToColumnar + 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] + 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,cs_ship_customer_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [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] + CometBroadcastExchange [ss_customer_sk] #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,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 InputAdapter - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] - CometFilter [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] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,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 + BroadcastExchange #6 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [cs_ship_customer_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,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 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) + BroadcastExchange #7 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,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 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) + BroadcastExchange #8 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/explain.txt index 93faa6420..c8d74de3d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (30) -+- * HashAggregate (29) - +- Exchange (28) - +- * ColumnarToRow (27) +* ColumnarToRow (30) ++- CometTakeOrderedAndProject (29) + +- CometHashAggregate (28) + +- CometColumnarExchange (27) +- CometHashAggregate (26) +- CometProject (25) +- CometBroadcastHashJoin (24) @@ -156,23 +156,21 @@ Input [5]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i 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))] -(27) ColumnarToRow [codegen id : 1] +(27) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(28) Exchange -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 : 2] +(28) CometHashAggregate 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)#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#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] +(29) CometTakeOrderedAndProject +Input [5]: [i_item_id#17, agg1#29, agg2#30, agg3#31, agg4#32] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#17 ASC NULLS FIRST], output=[i_item_id#17,agg1#29,agg2#30,agg3#31,agg4#32]), [i_item_id#17, agg1#29, agg2#30, agg3#31, agg4#32], 100, [i_item_id#17 ASC NULLS FIRST], [i_item_id#17, agg1#29, agg2#30, agg3#31, agg4#32] + +(30) ColumnarToRow [codegen id : 1] +Input [5]: [i_item_id#17, agg1#29, agg2#30, agg3#31, agg4#32] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/simplified.txt index f327d5b06..0725862ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/simplified.txt @@ -1,42 +1,40 @@ -TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,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,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,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,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] - CometFilter [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #5 - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [p_promo_sk] #6 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_email,p_channel_event] - CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] + CometHashAggregate [i_item_id,agg1,agg2,agg3,agg4,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))] + CometColumnarExchange [i_item_id] #1 + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,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,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,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,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] + CometProject [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [cd_demo_sk] #3 + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_item_id] #5 + CometFilter [i_item_sk,i_item_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange [p_promo_sk] #6 + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_email,p_channel_event] + 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-spark3_5/q70/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/explain.txt index 4c59048bd..c144adc12 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/explain.txt @@ -1,49 +1,53 @@ == Physical Plan == -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) - +- * 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) +TakeOrderedAndProject (49) ++- * Project (48) + +- Window (47) + +- * ColumnarToRow (46) + +- CometSort (45) + +- CometColumnarExchange (44) + +- RowToColumnar (43) + +- * HashAggregate (42) + +- * ColumnarToRow (41) + +- CometColumnarExchange (40) + +- RowToColumnar (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) + +- * ColumnarToRow (27) + +- CometSort (26) + +- CometHashAggregate (25) + +- CometColumnarExchange (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 @@ -86,7 +90,7 @@ Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight 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] +(9) ColumnarToRow [codegen id : 4] Input [2]: [ss_store_sk#1, ss_net_profit#2] (10) Scan parquet spark_catalog.default.store @@ -100,7 +104,7 @@ ReadSchema: struct Input [3]: [s_store_sk#7, s_county#8, s_state#9] Condition : isnotnull(s_store_sk#7) -(12) ColumnarToRow [codegen id : 4] +(12) ColumnarToRow [codegen id : 3] Input [3]: [s_store_sk#7, s_county#8, s_state#9] (13) Scan parquet spark_catalog.default.store_sales @@ -156,45 +160,43 @@ 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] +(24) CometColumnarExchange Input [2]: [s_state#15, sum#17] +Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(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] +(25) CometHashAggregate 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 +(26) CometSort +Input [3]: [s_state#15, _w0#18, s_state#15] +Arguments: [s_state#15, _w0#18, s_state#15], [s_state#15 ASC NULLS FIRST, _w0#18 DESC NULLS LAST] + +(27) ColumnarToRow [codegen id : 1] +Input [3]: [s_state#15, _w0#18, s_state#15] (28) WindowGroupLimit -Input [3]: [s_state#15, _w0#19, s_state#15] -Arguments: [s_state#15], [_w0#19 DESC NULLS LAST], rank(_w0#19), 5, Final +Input [3]: [s_state#15, _w0#18, s_state#15] +Arguments: [s_state#15], [_w0#18 DESC NULLS LAST], rank(_w0#18), 5, Final (29) 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] +Input [3]: [s_state#15, _w0#18, s_state#15] +Arguments: [rank(_w0#18) windowspecdefinition(s_state#15, _w0#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#19], [s_state#15], [_w0#18 DESC NULLS LAST] -(30) Filter [codegen id : 3] -Input [4]: [s_state#15, _w0#19, s_state#15, ranking#20] -Condition : (ranking#20 <= 5) +(30) Filter [codegen id : 2] +Input [4]: [s_state#15, _w0#18, s_state#15, ranking#19] +Condition : (ranking#19 <= 5) -(31) Project [codegen id : 3] +(31) Project [codegen id : 2] Output [1]: [s_state#15] -Input [4]: [s_state#15, _w0#19, s_state#15, ranking#20] +Input [4]: [s_state#15, _w0#18, s_state#15, ranking#19] (32) BroadcastExchange Input [1]: [s_state#15] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] -(33) BroadcastHashJoin [codegen id : 4] +(33) BroadcastHashJoin [codegen id : 3] Left keys [1]: [s_state#9] Right keys [1]: [s_state#15] Join type: LeftSemi @@ -204,87 +206,99 @@ Join condition: None 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] -(35) BroadcastHashJoin [codegen id : 5] +(35) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_store_sk#1] Right keys [1]: [s_store_sk#7] Join type: Inner Join condition: None -(36) Project [codegen id : 5] +(36) Project [codegen id : 4] 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] -(37) Expand [codegen id : 5] +(37) Expand [codegen id : 4] 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] +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#20, s_county#21, spark_grouping_id#22] -(38) 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] +(38) HashAggregate [codegen id : 4] +Input [4]: [ss_net_profit#2, s_state#20, s_county#21, spark_grouping_id#22] +Keys [3]: [s_state#20, s_county#21, spark_grouping_id#22] 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#23] +Results [4]: [s_state#20, s_county#21, spark_grouping_id#22, sum#24] + +(39) RowToColumnar +Input [4]: [s_state#20, s_county#21, spark_grouping_id#22, sum#24] -(39) 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=4] +(40) CometColumnarExchange +Input [4]: [s_state#20, s_county#21, spark_grouping_id#22, sum#24] +Arguments: hashpartitioning(s_state#20, s_county#21, spark_grouping_id#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(40) 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] +(41) ColumnarToRow [codegen id : 5] +Input [4]: [s_state#20, s_county#21, spark_grouping_id#22, sum#24] + +(42) HashAggregate [codegen id : 5] +Input [4]: [s_state#20, s_county#21, spark_grouping_id#22, sum#24] +Keys [3]: [s_state#20, s_county#21, spark_grouping_id#22] 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))#25] +Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#25,17,2) AS total_sum#26, s_state#20, s_county#21, (cast((shiftright(spark_grouping_id#22, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#22, 0) & 1) as tinyint)) AS lochierarchy#27, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#25,17,2) AS _w0#28, (cast((shiftright(spark_grouping_id#22, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#22, 0) & 1) as tinyint)) AS _w1#29, CASE WHEN (cast((shiftright(spark_grouping_id#22, 0) & 1) as tinyint) = 0) THEN s_state#20 END AS _w2#30] + +(43) RowToColumnar +Input [7]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, _w0#28, _w1#29, _w2#30] + +(44) CometColumnarExchange +Input [7]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, _w0#28, _w1#29, _w2#30] +Arguments: hashpartitioning(_w1#29, _w2#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(41) 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=5] +(45) CometSort +Input [7]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, _w0#28, _w1#29, _w2#30] +Arguments: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, _w0#28, _w1#29, _w2#30], [_w1#29 ASC NULLS FIRST, _w2#30 ASC NULLS FIRST, _w0#28 DESC NULLS LAST] -(42) 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 +(46) ColumnarToRow [codegen id : 6] +Input [7]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, _w0#28, _w1#29, _w2#30] -(43) 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] +(47) Window +Input [7]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, _w0#28, _w1#29, _w2#30] +Arguments: [rank(_w0#28) windowspecdefinition(_w1#29, _w2#30, _w0#28 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#31], [_w1#29, _w2#30], [_w0#28 DESC NULLS LAST] -(44) 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] +(48) Project [codegen id : 7] +Output [5]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, rank_within_parent#31] +Input [8]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, _w0#28, _w1#29, _w2#30, rank_within_parent#31] -(45) 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] +(49) TakeOrderedAndProject +Input [5]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, rank_within_parent#31] +Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN s_state#20 END ASC NULLS FIRST, rank_within_parent#31 ASC NULLS FIRST], [total_sum#26, s_state#20, s_county#21, lochierarchy#27, rank_within_parent#31] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (50) -+- * ColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan parquet spark_catalog.default.date_dim (46) +BroadcastExchange (54) ++- * ColumnarToRow (53) + +- CometProject (52) + +- CometFilter (51) + +- CometScan parquet spark_catalog.default.date_dim (50) -(46) Scan parquet spark_catalog.default.date_dim +(50) 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 -(47) CometFilter +(51) 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)) -(48) CometProject +(52) CometProject Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(49) ColumnarToRow [codegen id : 1] +(53) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(50) BroadcastExchange +(54) BroadcastExchange Input [1]: [d_date_sk#5] 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-spark3_5/q70/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/simplified.txt index 030dfc4d8..95a617c5a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/simplified.txt @@ -1,72 +1,74 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (8) + WholeStageCodegen (7) Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] InputAdapter Window [_w0,_w1,_w2] - WholeStageCodegen (7) - Sort [_w1,_w2,_w0] + WholeStageCodegen (6) + ColumnarToRow InputAdapter - Exchange [_w1,_w2] #1 - 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 (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] - ColumnarToRow - InputAdapter - CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - BroadcastHashJoin [s_state,s_state] + CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] + CometColumnarExchange [_w1,_w2] #1 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 + RowToColumnar + WholeStageCodegen (4) + 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] ColumnarToRow InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + CometProject [ss_store_sk,ss_net_profit] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - BroadcastExchange #6 + BroadcastExchange #5 WholeStageCodegen (3) - Project [s_state] - Filter [ranking] + BroadcastHashJoin [s_state,s_state] + ColumnarToRow InputAdapter - Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (2) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - InputAdapter - Exchange [s_state] #7 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [s_state,sum,ss_net_profit] - CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] - CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #8 - CometFilter [s_store_sk,s_state] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [d_date_sk] #4 + CometFilter [s_store_sk,s_county,s_state] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + Project [s_state] + Filter [ranking] + InputAdapter + Window [_w0,s_state] + WindowGroupLimit [s_state,_w0] + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometSort [s_state,_w0] + CometHashAggregate [s_state,_w0,sum,sum(UnscaledValue(ss_net_profit))] + CometColumnarExchange [s_state] #7 + CometHashAggregate [s_state,sum,ss_net_profit] + CometProject [ss_net_profit,s_state] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] + CometProject [ss_net_profit,ss_sold_date_sk,s_state] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [s_store_sk,s_state] #8 + CometFilter [s_store_sk,s_state] + 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-spark3_5/q71/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/explain.txt index f2128cfff..53691edc5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -* Sort (37) -+- Exchange (36) - +- * HashAggregate (35) - +- Exchange (34) - +- * ColumnarToRow (33) +* ColumnarToRow (37) ++- CometSort (36) + +- CometColumnarExchange (35) + +- CometHashAggregate (34) + +- CometColumnarExchange (33) +- CometHashAggregate (32) +- CometProject (31) +- CometBroadcastHashJoin (30) @@ -192,27 +192,25 @@ 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] +(33) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(34) CometHashAggregate 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] +(35) CometColumnarExchange +Input [5]: [brand_id#39, brand#40, t_hour#35, t_minute#36, ext_price#41] +Arguments: rangepartitioning(ext_price#41 DESC NULLS LAST, brand_id#39 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(36) CometSort +Input [5]: [brand_id#39, brand#40, t_hour#35, t_minute#36, ext_price#41] +Arguments: [brand_id#39, brand#40, t_hour#35, t_minute#36, ext_price#41], [ext_price#41 DESC NULLS LAST, brand_id#39 ASC NULLS FIRST] -(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 +(37) ColumnarToRow [codegen id : 1] +Input [5]: [brand_id#39, brand#40, t_hour#35, t_minute#36, ext_price#41] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/simplified.txt index c604a8fa7..cd0700c28 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/simplified.txt @@ -1,53 +1,49 @@ -WholeStageCodegen (3) - Sort [ext_price,brand_id] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [ext_price,brand_id] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum,ext_price] - CometProject [i_brand_id,i_brand,ext_price,t_hour,t_minute] - CometBroadcastHashJoin [i_brand_id,i_brand,ext_price,time_sk,t_time_sk,t_hour,t_minute] - CometProject [i_brand_id,i_brand,ext_price,time_sk] - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_brand,ext_price,sold_item_sk,time_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometUnion [ext_price,sold_item_sk,time_sk] - CometProject [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] - CometFilter [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - 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_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] - CometFilter [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometFilter [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_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 [t_time_sk,t_hour,t_minute] #6 - CometProject [t_time_sk,t_hour,t_minute] - CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] + CometSort [brand_id,brand,t_hour,t_minute,ext_price] + CometColumnarExchange [ext_price,brand_id] #1 + CometHashAggregate [brand_id,brand,t_hour,t_minute,ext_price,i_brand,i_brand_id,sum,sum(UnscaledValue(ext_price))] + CometColumnarExchange [i_brand,i_brand_id,t_hour,t_minute] #2 + CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum,ext_price] + CometProject [i_brand_id,i_brand,ext_price,t_hour,t_minute] + CometBroadcastHashJoin [i_brand_id,i_brand,ext_price,time_sk,t_time_sk,t_hour,t_minute] + CometProject [i_brand_id,i_brand,ext_price,time_sk] + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_brand,ext_price,sold_item_sk,time_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 + CometProject [i_item_sk,i_brand_id,i_brand] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] + CometUnion [ext_price,sold_item_sk,time_sk] + CometProject [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] [ext_price,sold_item_sk,time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_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_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + 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_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_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 [t_time_sk,t_hour,t_minute] #6 + CometProject [t_time_sk,t_hour,t_minute] + CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] + 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-spark3_5/q72/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/explain.txt index 26edd145c..2f6da7d32 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/explain.txt @@ -1,71 +1,72 @@ == Physical Plan == -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) +* ColumnarToRow (68) ++- CometTakeOrderedAndProject (67) + +- CometHashAggregate (66) + +- CometColumnarExchange (65) + +- CometHashAggregate (64) + +- CometProject (63) + +- CometSortMergeJoin (62) + :- CometSort (56) + : +- CometColumnarExchange (55) + : +- RowToColumnar (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) + +- CometSort (61) + +- CometColumnarExchange (60) + +- CometProject (59) + +- CometFilter (58) + +- CometScan parquet spark_catalog.default.catalog_returns (57) (1) Scan parquet spark_catalog.default.catalog_sales @@ -320,101 +321,99 @@ Join condition: None 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#30] -(54) Exchange +(54) RowToColumnar 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=3] -(55) Sort [codegen id : 4] +(55) CometColumnarExchange 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 +Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(56) Scan parquet spark_catalog.default.catalog_returns +(56) CometSort +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, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST] + +(57) 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 -(57) CometFilter +(58) 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)) -(58) CometProject +(59) 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] -(59) ColumnarToRow [codegen id : 5] +(60) CometColumnarExchange Input [2]: [cr_item_sk#31, cr_order_number#32] +Arguments: hashpartitioning(cr_item_sk#31, cr_order_number#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(60) Exchange +(61) CometSort 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] +Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31 ASC NULLS FIRST, cr_order_number#32 ASC NULLS FIRST] -(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 +(62) CometSortMergeJoin +Left output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +Right output [2]: [cr_item_sk#31, cr_order_number#32] +Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#31, cr_order_number#32], LeftOuter -(62) SortMergeJoin [codegen id : 7] -Left keys [2]: [cs_item_sk#4, cs_order_number#6] -Right keys [2]: [cr_item_sk#31, cr_order_number#32] -Join type: LeftOuter -Join condition: None - -(63) Project [codegen id : 7] -Output [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +(63) CometProject 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] +Arguments: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -(64) HashAggregate [codegen id : 7] +(64) CometHashAggregate 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#34] -Results [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#35] -(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] +(65) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=5] -(66) HashAggregate [codegen id : 8] -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#35] +(66) CometHashAggregate +Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] Functions [1]: [count(1)] -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] -(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] +(67) CometTakeOrderedAndProject +Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#37 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#35,promo#36,total_cnt#37]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37], 100, [total_cnt#37 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#35, promo#36, total_cnt#37] + +(68) ColumnarToRow [codegen id : 4] +Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -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 +(69) 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 -(69) CometFilter +(70) 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)) -(70) CometProject +(71) 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] -(71) ColumnarToRow [codegen id : 1] +(72) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -(72) BroadcastExchange +(73) 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=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/simplified.txt index 6cba2d0e6..1022e1a90 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/simplified.txt @@ -1,94 +1,86 @@ -TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo] - 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 (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 (4) - Sort [cs_item_sk,cs_order_number] - InputAdapter - Exchange [cs_item_sk,cs_order_number] #2 - 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] - 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 [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,d_date_sk,d_week_seq] - 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_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,d_date_sk,d_date,d_week_seq] - 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_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,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_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,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_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_sk,i_item_desc] - 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 [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,w_warehouse_sk,w_warehouse_name] - 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_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,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometFilter [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] - 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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [i_item_sk,i_item_desc] #6 - CometFilter [i_item_sk,i_item_desc] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange [cd_demo_sk] #7 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange [hd_demo_sk] #8 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange [d_date_sk,d_week_seq] #10 - CometFilter [d_date_sk,d_week_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] +WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] + CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count,count(1)] + CometColumnarExchange [i_item_desc,w_warehouse_name,d_week_seq] #1 + CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] + CometProject [w_warehouse_name,i_item_desc,d_week_seq] + CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] + CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + CometColumnarExchange [cs_item_sk,cs_order_number] #2 + RowToColumnar + 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] + 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 [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,d_date_sk,d_week_seq] + 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_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,d_date_sk,d_date,d_week_seq] + 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_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,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_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,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_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_sk,i_item_desc] + 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 [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,w_warehouse_sk,w_warehouse_name] + 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_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,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometFilter [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] + 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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange [i_item_sk,i_item_desc] #6 + CometFilter [i_item_sk,i_item_desc] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + CometBroadcastExchange [cd_demo_sk] #7 + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + CometBroadcastExchange [hd_demo_sk] #8 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] + CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + CometBroadcastExchange [d_date_sk,d_week_seq] #10 + CometFilter [d_date_sk,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter - BroadcastExchange #12 - WholeStageCodegen (2) + BroadcastExchange #11 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk] - InputAdapter - WholeStageCodegen (6) - Sort [cr_item_sk,cr_order_number] - InputAdapter - Exchange [cr_item_sk,cr_order_number] #13 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk] + CometSort [cr_item_sk,cr_order_number] + CometColumnarExchange [cr_item_sk,cr_order_number] #13 + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/explain.txt index 88dcba0fb..6ea426978 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/explain.txt @@ -1,37 +1,36 @@ == Physical Plan == -* 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) +* ColumnarToRow (32) ++- CometSort (31) + +- CometColumnarExchange (30) + +- CometProject (29) + +- CometBroadcastHashJoin (28) + :- CometFilter (24) + : +- CometHashAggregate (23) + : +- CometColumnarExchange (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) + +- CometBroadcastExchange (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.customer (25) (1) Scan parquet spark_catalog.default.store_sales @@ -135,90 +134,84 @@ 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)] -(22) ColumnarToRow [codegen id : 1] +(22) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(24) HashAggregate [codegen id : 3] +(23) CometHashAggregate 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)#17] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#17 AS cnt#18] -(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)) +(24) CometFilter +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] +Condition : ((cnt#17 >= 1) AND (cnt#17 <= 5)) -(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] +(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] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(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) +(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) -(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] +(27) CometBroadcastExchange +Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Arguments: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -(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] +(28) CometBroadcastHashJoin +Left output [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] +Right output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight -(30) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#19] -Join type: Inner -Join condition: None +(29) CometProject +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] +Arguments: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17], [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -(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] +(30) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] -(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] +(31) CometSort +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, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17], [cnt#17 DESC NULLS LAST] -(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 +(32) ColumnarToRow [codegen id : 1] +Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (38) -+- * ColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.date_dim (34) +BroadcastExchange (37) ++- * ColumnarToRow (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) -(34) Scan parquet spark_catalog.default.date_dim +(33) 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 -(35) CometFilter +(34) 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)) -(36) CometProject +(35) CometProject Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(37) ColumnarToRow [codegen id : 1] +(36) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(38) BroadcastExchange +(37) BroadcastExchange Input [1]: [d_date_sk#7] -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-spark3_5/q73/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/simplified.txt index c91c4cf47..5fb8a197a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/simplified.txt @@ -1,50 +1,42 @@ -WholeStageCodegen (4) - Sort [cnt] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [cnt] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,count] - CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] - CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] - CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #5 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange [hd_demo_sk] #6 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + CometColumnarExchange [cnt] #1 + CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometFilter [ss_ticket_number,ss_customer_sk,cnt] + CometHashAggregate [ss_ticket_number,ss_customer_sk,cnt,count,count(1)] + CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] + CometProject [ss_customer_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange [s_store_sk] #5 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_county] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county] + CometBroadcastExchange [hd_demo_sk] #6 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] #7 + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/explain.txt index 83e2737fd..ecc0fd1ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/explain.txt @@ -1,75 +1,72 @@ == Physical Plan == -TakeOrderedAndProject (71) -+- * Project (70) - +- * BroadcastHashJoin Inner BuildRight (69) - :- * 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) - +- * 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) +* ColumnarToRow (68) ++- CometTakeOrderedAndProject (67) + +- CometProject (66) + +- CometBroadcastHashJoin (65) + :- CometProject (50) + : +- CometBroadcastHashJoin (49) + : :- CometBroadcastHashJoin (33) + : : :- CometFilter (16) + : : : +- CometHashAggregate (15) + : : : +- CometColumnarExchange (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) + : : +- CometBroadcastExchange (32) + : : +- CometHashAggregate (31) + : : +- CometColumnarExchange (30) + : : +- CometHashAggregate (29) + : : +- CometProject (28) + : : +- CometBroadcastHashJoin (27) + : : :- CometProject (23) + : : : +- CometBroadcastHashJoin (22) + : : : :- CometFilter (18) + : : : : +- CometScan parquet spark_catalog.default.customer (17) + : : : +- CometBroadcastExchange (21) + : : : +- CometFilter (20) + : : : +- CometScan parquet spark_catalog.default.store_sales (19) + : : +- CometBroadcastExchange (26) + : : +- CometFilter (25) + : : +- CometScan parquet spark_catalog.default.date_dim (24) + : +- CometBroadcastExchange (48) + : +- CometFilter (47) + : +- CometHashAggregate (46) + : +- CometColumnarExchange (45) + : +- CometHashAggregate (44) + : +- CometProject (43) + : +- CometBroadcastHashJoin (42) + : :- CometProject (40) + : : +- CometBroadcastHashJoin (39) + : : :- CometFilter (35) + : : : +- CometScan parquet spark_catalog.default.customer (34) + : : +- CometBroadcastExchange (38) + : : +- CometFilter (37) + : : +- CometScan parquet spark_catalog.default.web_sales (36) + : +- ReusedExchange (41) + +- CometBroadcastExchange (64) + +- CometHashAggregate (63) + +- CometColumnarExchange (62) + +- CometHashAggregate (61) + +- CometProject (60) + +- CometBroadcastHashJoin (59) + :- CometProject (57) + : +- CometBroadcastHashJoin (56) + : :- CometFilter (52) + : : +- CometScan parquet spark_catalog.default.customer (51) + : +- CometBroadcastExchange (55) + : +- CometFilter (54) + : +- CometScan parquet spark_catalog.default.web_sales (53) + +- ReusedExchange (58) (1) Scan parquet spark_catalog.default.customer @@ -137,337 +134,317 @@ Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_yea 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))] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(16) HashAggregate [codegen id : 8] +(15) CometHashAggregate 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))#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] -(17) Filter [codegen id : 8] -Input [2]: [customer_id#13, year_total#14] -Condition : (isnotnull(year_total#14) AND (year_total#14 > 0.00)) +(16) CometFilter +Input [2]: [customer_id#12, year_total#13] +Condition : (isnotnull(year_total#13) AND (year_total#13 > 0.00)) -(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] +(17) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#14, c_customer_id#15, c_first_name#16, c_last_name#17] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(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)) +(18) CometFilter +Input [4]: [c_customer_sk#14, c_customer_id#15, c_first_name#16, c_last_name#17] +Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_customer_id#15)) -(20) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] +(19) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_customer_sk#18, ss_net_paid#19, ss_sold_date_sk#20] 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#20), dynamicpruningexpression(ss_sold_date_sk#20 IN dynamicpruning#21)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(21) CometFilter -Input [3]: [ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] -Condition : isnotnull(ss_customer_sk#19) +(20) CometFilter +Input [3]: [ss_customer_sk#18, ss_net_paid#19, ss_sold_date_sk#20] +Condition : isnotnull(ss_customer_sk#18) -(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] +(21) CometBroadcastExchange +Input [3]: [ss_customer_sk#18, ss_net_paid#19, ss_sold_date_sk#20] +Arguments: [ss_customer_sk#18, ss_net_paid#19, ss_sold_date_sk#20] -(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 +(22) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#14, c_customer_id#15, c_first_name#16, c_last_name#17] +Right output [3]: [ss_customer_sk#18, ss_net_paid#19, ss_sold_date_sk#20] +Arguments: [c_customer_sk#14], [ss_customer_sk#18], Inner, BuildRight -(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] +(23) CometProject +Input [7]: [c_customer_sk#14, c_customer_id#15, c_first_name#16, c_last_name#17, ss_customer_sk#18, ss_net_paid#19, ss_sold_date_sk#20] +Arguments: [c_customer_id#15, c_first_name#16, c_last_name#17, ss_net_paid#19, ss_sold_date_sk#20], [c_customer_id#15, c_first_name#16, c_last_name#17, ss_net_paid#19, ss_sold_date_sk#20] -(25) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#23, d_year#24] +(24) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#22, d_year#23] 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 -(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 - -(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] +(25) CometFilter +Input [2]: [d_date_sk#22, d_year#23] +Condition : (((isnotnull(d_year#23) AND (d_year#23 = 2002)) AND d_year#23 IN (2001,2002)) AND isnotnull(d_date_sk#22)) + +(26) CometBroadcastExchange +Input [2]: [d_date_sk#22, d_year#23] +Arguments: [d_date_sk#22, d_year#23] + +(27) CometBroadcastHashJoin +Left output [5]: [c_customer_id#15, c_first_name#16, c_last_name#17, ss_net_paid#19, ss_sold_date_sk#20] +Right output [2]: [d_date_sk#22, d_year#23] +Arguments: [ss_sold_date_sk#20], [d_date_sk#22], Inner, BuildRight + +(28) CometProject +Input [7]: [c_customer_id#15, c_first_name#16, c_last_name#17, ss_net_paid#19, ss_sold_date_sk#20, d_date_sk#22, d_year#23] +Arguments: [c_customer_id#15, c_first_name#16, c_last_name#17, ss_net_paid#19, d_year#23], [c_customer_id#15, c_first_name#16, c_last_name#17, ss_net_paid#19, d_year#23] + +(29) CometHashAggregate +Input [5]: [c_customer_id#15, c_first_name#16, c_last_name#17, ss_net_paid#19, d_year#23] +Keys [4]: [c_customer_id#15, c_first_name#16, c_last_name#17, d_year#23] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#19))] + +(30) CometColumnarExchange +Input [5]: [c_customer_id#15, c_first_name#16, c_last_name#17, d_year#23, sum#24] +Arguments: hashpartitioning(c_customer_id#15, c_first_name#16, c_last_name#17, d_year#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(31) CometHashAggregate +Input [5]: [c_customer_id#15, c_first_name#16, c_last_name#17, d_year#23, sum#24] +Keys [4]: [c_customer_id#15, c_first_name#16, c_last_name#17, d_year#23] +Functions [1]: [sum(UnscaledValue(ss_net_paid#19))] + +(32) CometBroadcastExchange +Input [4]: [customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28] +Arguments: [customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28] + +(33) CometBroadcastHashJoin +Left output [2]: [customer_id#12, year_total#13] +Right output [4]: [customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28] +Arguments: [customer_id#12], [customer_id#25], Inner, BuildRight + +(34) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#29, c_customer_id#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_customer_id)] ReadSchema: struct -(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)) +(35) CometFilter +Input [4]: [c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] +Condition : (isnotnull(c_customer_sk#29) AND isnotnull(c_customer_id#30)) -(38) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#34, ws_net_paid#35, ws_sold_date_sk#36] +(36) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#37)] +PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#36)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(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) +(37) CometFilter +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Condition : isnotnull(ws_bill_customer_sk#33) + +(38) CometBroadcastExchange +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Arguments: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] -(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] +(39) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] +Right output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Arguments: [c_customer_sk#29], [ws_bill_customer_sk#33], Inner, BuildRight -(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 +(40) CometProject +Input [7]: [c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32, ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Arguments: [c_customer_id#30, c_first_name#31, c_last_name#32, ws_net_paid#34, ws_sold_date_sk#35], [c_customer_id#30, c_first_name#31, c_last_name#32, ws_net_paid#34, ws_sold_date_sk#35] -(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] +(41) ReusedExchange [Reuses operator id: 10] +Output [2]: [d_date_sk#37, d_year#38] -(43) ReusedExchange [Reuses operator id: 10] -Output [2]: [d_date_sk#38, d_year#39] +(42) CometBroadcastHashJoin +Left output [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, ws_net_paid#34, ws_sold_date_sk#35] +Right output [2]: [d_date_sk#37, d_year#38] +Arguments: [ws_sold_date_sk#35], [d_date_sk#37], Inner, BuildRight -(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 +(43) CometProject +Input [7]: [c_customer_id#30, c_first_name#31, c_last_name#32, ws_net_paid#34, ws_sold_date_sk#35, d_date_sk#37, d_year#38] +Arguments: [c_customer_id#30, c_first_name#31, c_last_name#32, ws_net_paid#34, d_year#38], [c_customer_id#30, c_first_name#31, c_last_name#32, ws_net_paid#34, d_year#38] -(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] +(44) CometHashAggregate +Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, ws_net_paid#34, d_year#38] +Keys [4]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#38] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#34))] + +(45) CometColumnarExchange +Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#38, sum#39] +Arguments: hashpartitioning(c_customer_id#30, c_first_name#31, c_last_name#32, d_year#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (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] - -(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 - -(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] - -(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] +Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#38, sum#39] +Keys [4]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#38] +Functions [1]: [sum(UnscaledValue(ws_net_paid#34))] + +(47) CometFilter +Input [2]: [customer_id#40, year_total#41] +Condition : (isnotnull(year_total#41) AND (year_total#41 > 0.00)) + +(48) CometBroadcastExchange +Input [2]: [customer_id#40, year_total#41] +Arguments: [customer_id#40, year_total#41] + +(49) CometBroadcastHashJoin +Left output [6]: [customer_id#12, year_total#13, customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28] +Right output [2]: [customer_id#40, year_total#41] +Arguments: [customer_id#12], [customer_id#40], Inner, BuildRight + +(50) CometProject +Input [8]: [customer_id#12, year_total#13, customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28, customer_id#40, year_total#41] +Arguments: [customer_id#12, year_total#13, customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28, year_total#41], [customer_id#12, year_total#13, customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28, year_total#41] + +(51) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#42, c_customer_id#43, c_first_name#44, c_last_name#45] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(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)) +(52) CometFilter +Input [4]: [c_customer_sk#42, c_customer_id#43, c_first_name#44, c_last_name#45] +Condition : (isnotnull(c_customer_sk#42) AND isnotnull(c_customer_id#43)) -(56) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] +(53) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_customer_sk#46, ws_net_paid#47, ws_sold_date_sk#48] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#50), dynamicpruningexpression(ws_sold_date_sk#50 IN dynamicpruning#51)] +PartitionFilters: [isnotnull(ws_sold_date_sk#48), dynamicpruningexpression(ws_sold_date_sk#48 IN dynamicpruning#49)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(57) CometFilter -Input [3]: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] -Condition : isnotnull(ws_bill_customer_sk#48) +(54) CometFilter +Input [3]: [ws_bill_customer_sk#46, ws_net_paid#47, ws_sold_date_sk#48] +Condition : isnotnull(ws_bill_customer_sk#46) -(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] +(55) CometBroadcastExchange +Input [3]: [ws_bill_customer_sk#46, ws_net_paid#47, ws_sold_date_sk#48] +Arguments: [ws_bill_customer_sk#46, ws_net_paid#47, ws_sold_date_sk#48] + +(56) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#42, c_customer_id#43, c_first_name#44, c_last_name#45] +Right output [3]: [ws_bill_customer_sk#46, ws_net_paid#47, ws_sold_date_sk#48] +Arguments: [c_customer_sk#42], [ws_bill_customer_sk#46], Inner, BuildRight + +(57) CometProject +Input [7]: [c_customer_sk#42, c_customer_id#43, c_first_name#44, c_last_name#45, ws_bill_customer_sk#46, ws_net_paid#47, ws_sold_date_sk#48] +Arguments: [c_customer_id#43, c_first_name#44, c_last_name#45, ws_net_paid#47, ws_sold_date_sk#48], [c_customer_id#43, c_first_name#44, c_last_name#45, ws_net_paid#47, ws_sold_date_sk#48] + +(58) ReusedExchange [Reuses operator id: 26] +Output [2]: [d_date_sk#50, d_year#51] (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 +Left output [5]: [c_customer_id#43, c_first_name#44, c_last_name#45, ws_net_paid#47, ws_sold_date_sk#48] +Right output [2]: [d_date_sk#50, d_year#51] +Arguments: [ws_sold_date_sk#48], [d_date_sk#50], Inner, BuildRight (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) ReusedExchange [Reuses operator id: 27] -Output [2]: [d_date_sk#52, d_year#53] - -(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) 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) 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) 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#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 : 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#55, year_total#56] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] - -(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#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 : 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#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] +Input [7]: [c_customer_id#43, c_first_name#44, c_last_name#45, ws_net_paid#47, ws_sold_date_sk#48, d_date_sk#50, d_year#51] +Arguments: [c_customer_id#43, c_first_name#44, c_last_name#45, ws_net_paid#47, d_year#51], [c_customer_id#43, c_first_name#44, c_last_name#45, ws_net_paid#47, d_year#51] + +(61) CometHashAggregate +Input [5]: [c_customer_id#43, c_first_name#44, c_last_name#45, ws_net_paid#47, d_year#51] +Keys [4]: [c_customer_id#43, c_first_name#44, c_last_name#45, d_year#51] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#47))] + +(62) CometColumnarExchange +Input [5]: [c_customer_id#43, c_first_name#44, c_last_name#45, d_year#51, sum#52] +Arguments: hashpartitioning(c_customer_id#43, c_first_name#44, c_last_name#45, d_year#51, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(63) CometHashAggregate +Input [5]: [c_customer_id#43, c_first_name#44, c_last_name#45, d_year#51, sum#52] +Keys [4]: [c_customer_id#43, c_first_name#44, c_last_name#45, d_year#51] +Functions [1]: [sum(UnscaledValue(ws_net_paid#47))] + +(64) CometBroadcastExchange +Input [2]: [customer_id#53, year_total#54] +Arguments: [customer_id#53, year_total#54] + +(65) CometBroadcastHashJoin +Left output [7]: [customer_id#12, year_total#13, customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28, year_total#41] +Right output [2]: [customer_id#53, year_total#54] +Arguments: [customer_id#12], [customer_id#53], Inner, (CASE WHEN (year_total#41 > 0.00) THEN (year_total#54 / year_total#41) END > CASE WHEN (year_total#13 > 0.00) THEN (year_total#28 / year_total#13) END), BuildRight + +(66) CometProject +Input [9]: [customer_id#12, year_total#13, customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28, year_total#41, customer_id#53, year_total#54] +Arguments: [customer_id#25, customer_first_name#26, customer_last_name#27], [customer_id#25, customer_first_name#26, customer_last_name#27] + +(67) CometTakeOrderedAndProject +Input [3]: [customer_id#25, customer_first_name#26, customer_last_name#27] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#25 ASC NULLS FIRST,customer_id#25 ASC NULLS FIRST,customer_id#25 ASC NULLS FIRST], output=[customer_id#25,customer_first_name#26,customer_last_name#27]), [customer_id#25, customer_first_name#26, customer_last_name#27], 100, [customer_id#25 ASC NULLS FIRST, customer_id#25 ASC NULLS FIRST, customer_id#25 ASC NULLS FIRST], [customer_id#25, customer_first_name#26, customer_last_name#27] + +(68) ColumnarToRow [codegen id : 1] +Input [3]: [customer_id#25, customer_first_name#26, customer_last_name#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (75) -+- * ColumnarToRow (74) - +- CometFilter (73) - +- CometScan parquet spark_catalog.default.date_dim (72) +BroadcastExchange (72) ++- * ColumnarToRow (71) + +- CometFilter (70) + +- CometScan parquet spark_catalog.default.date_dim (69) -(72) Scan parquet spark_catalog.default.date_dim +(69) 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 -(73) CometFilter +(70) 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)) -(74) ColumnarToRow [codegen id : 1] +(71) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_year#10] -(75) BroadcastExchange +(72) BroadcastExchange Input [2]: [d_date_sk#9, d_year#10] -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=5] -Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#21 IN dynamicpruning#22 -BroadcastExchange (79) -+- * ColumnarToRow (78) - +- CometFilter (77) - +- CometScan parquet spark_catalog.default.date_dim (76) +Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#21 +BroadcastExchange (76) ++- * ColumnarToRow (75) + +- CometFilter (74) + +- CometScan parquet spark_catalog.default.date_dim (73) -(76) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#23, d_year#24] +(73) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#22, d_year#23] 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#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)) +(74) CometFilter +Input [2]: [d_date_sk#22, d_year#23] +Condition : (((isnotnull(d_year#23) AND (d_year#23 = 2002)) AND d_year#23 IN (2001,2002)) AND isnotnull(d_date_sk#22)) -(78) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#23, d_year#24] +(75) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#22, d_year#23] -(79) BroadcastExchange -Input [2]: [d_date_sk#23, d_year#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] +(76) BroadcastExchange +Input [2]: [d_date_sk#22, d_year#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -Subquery:3 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#8 -Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#50 IN dynamicpruning#22 +Subquery:4 Hosting operator id = 53 Hosting Expression = ws_sold_date_sk#48 IN dynamicpruning#21 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/simplified.txt index 7c2a42ca3..0b288df94 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/simplified.txt @@ -1,106 +1,86 @@ -TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] - 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] - BroadcastHashJoin [customer_id,customer_id] - BroadcastHashJoin [customer_id,customer_id] - Filter [year_total] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_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_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - 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,sum,ss_net_paid] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - 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] #11 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 - CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_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 #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] #14 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 - CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_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 +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] + CometProject [customer_id,customer_first_name,customer_last_name] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total,customer_id,year_total] + CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,customer_id,year_total] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total] + CometFilter [customer_id,year_total] + CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,d_year,sum,sum(UnscaledValue(ss_net_paid))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_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_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,year_total] #5 + CometHashAggregate [customer_id,customer_first_name,customer_last_name,year_total,c_customer_id,c_first_name,c_last_name,d_year,sum,sum(UnscaledValue(ss_net_paid))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #6 + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [customer_id,year_total] #10 + CometFilter [customer_id,year_total] + CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,d_year,sum,sum(UnscaledValue(ws_net_paid))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #11 + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_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 + CometBroadcastExchange [customer_id,year_total] #13 + CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,d_year,sum,sum(UnscaledValue(ws_net_paid))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #14 + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_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-spark3_5/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/explain.txt index 1f736c62c..ca9a5cd59 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/explain.txt @@ -1,136 +1,129 @@ == Physical Plan == -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) +TakeOrderedAndProject (125) ++- * Project (124) + +- * SortMergeJoin Inner (123) + :- * ColumnarToRow (67) + : +- CometSort (66) + : +- CometColumnarExchange (65) + : +- CometFilter (64) + : +- CometHashAggregate (63) + : +- CometColumnarExchange (62) + : +- CometHashAggregate (61) + : +- CometHashAggregate (60) + : +- CometColumnarExchange (59) + : +- CometHashAggregate (58) + : +- CometUnion (57) + : :- CometProject (22) + : : +- CometSortMergeJoin (21) + : : :- CometSort (15) + : : : +- CometColumnarExchange (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) + : : +- CometSort (20) + : : +- CometColumnarExchange (19) + : : +- CometProject (18) + : : +- CometFilter (17) + : : +- CometScan parquet spark_catalog.default.catalog_returns (16) + : :- CometProject (39) + : : +- CometSortMergeJoin (38) + : : :- CometSort (32) + : : : +- CometColumnarExchange (31) + : : : +- CometProject (30) + : : : +- CometBroadcastHashJoin (29) + : : : :- CometProject (27) + : : : : +- CometBroadcastHashJoin (26) + : : : : :- CometFilter (24) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (23) + : : : : +- ReusedExchange (25) + : : : +- ReusedExchange (28) + : : +- CometSort (37) + : : +- CometColumnarExchange (36) + : : +- CometProject (35) + : : +- CometFilter (34) + : : +- CometScan parquet spark_catalog.default.store_returns (33) + : +- CometProject (56) + : +- CometSortMergeJoin (55) + : :- CometSort (49) + : : +- CometColumnarExchange (48) + : : +- CometProject (47) + : : +- CometBroadcastHashJoin (46) + : : :- CometProject (44) + : : : +- CometBroadcastHashJoin (43) + : : : :- CometFilter (41) + : : : : +- CometScan parquet spark_catalog.default.web_sales (40) + : : : +- ReusedExchange (42) + : : +- ReusedExchange (45) + : +- CometSort (54) + : +- CometColumnarExchange (53) + : +- CometProject (52) + : +- CometFilter (51) + : +- CometScan parquet spark_catalog.default.web_returns (50) + +- * ColumnarToRow (122) + +- CometSort (121) + +- CometColumnarExchange (120) + +- CometFilter (119) + +- CometHashAggregate (118) + +- CometColumnarExchange (117) + +- CometHashAggregate (116) + +- CometHashAggregate (115) + +- CometColumnarExchange (114) + +- CometHashAggregate (113) + +- CometUnion (112) + :- CometProject (83) + : +- CometSortMergeJoin (82) + : :- CometSort (79) + : : +- CometColumnarExchange (78) + : : +- CometProject (77) + : : +- CometBroadcastHashJoin (76) + : : :- CometProject (72) + : : : +- CometBroadcastHashJoin (71) + : : : :- CometFilter (69) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (68) + : : : +- ReusedExchange (70) + : : +- CometBroadcastExchange (75) + : : +- CometFilter (74) + : : +- CometScan parquet spark_catalog.default.date_dim (73) + : +- CometSort (81) + : +- ReusedExchange (80) + :- CometProject (97) + : +- CometSortMergeJoin (96) + : :- CometSort (93) + : : +- CometColumnarExchange (92) + : : +- CometProject (91) + : : +- CometBroadcastHashJoin (90) + : : :- CometProject (88) + : : : +- CometBroadcastHashJoin (87) + : : : :- CometFilter (85) + : : : : +- CometScan parquet spark_catalog.default.store_sales (84) + : : : +- ReusedExchange (86) + : : +- ReusedExchange (89) + : +- CometSort (95) + : +- ReusedExchange (94) + +- CometProject (111) + +- CometSortMergeJoin (110) + :- CometSort (107) + : +- CometColumnarExchange (106) + : +- CometProject (105) + : +- CometBroadcastHashJoin (104) + : :- CometProject (102) + : : +- CometBroadcastHashJoin (101) + : : :- CometFilter (99) + : : : +- CometScan parquet spark_catalog.default.web_sales (98) + : : +- ReusedExchange (100) + : +- ReusedExchange (103) + +- CometSort (109) + +- ReusedExchange (108) (1) Scan parquet spark_catalog.default.catalog_sales @@ -197,54 +190,47 @@ Arguments: [cs_sold_date_sk#5], [d_date_sk#13], Inner, BuildRight 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] +(14) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(15) Exchange +(15) CometSort 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=1] +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_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST] -(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 - -(17) Scan parquet spark_catalog.default.catalog_returns +(16) 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 -(18) CometFilter +(17) 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)) -(19) CometProject +(18) 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] -(20) ColumnarToRow [codegen id : 3] -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] - -(21) Exchange +(19) CometColumnarExchange 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=2] +Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(22) Sort [codegen id : 4] +(20) CometSort 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 +Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST] -(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 +(21) CometSortMergeJoin +Left 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] +Right output [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] +Arguments: [cs_order_number#2, cs_item_sk#1], [cr_order_number#16, cr_item_sk#15], LeftOuter -(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] +(22) CometProject 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] +Arguments: [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], [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] -(25) Scan parquet spark_catalog.default.store_sales +(23) 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 [] @@ -252,82 +238,75 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(26) CometFilter +(24) 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) -(27) ReusedExchange [Reuses operator id: 6] +(25) 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) CometBroadcastHashJoin +(26) 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) CometProject +(27) 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: 11] +(28) ReusedExchange [Reuses operator id: 11] Output [2]: [d_date_sk#33, d_year#34] -(31) CometBroadcastHashJoin +(29) 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) CometProject +(30) 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] +(31) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=3] -(34) Exchange +(32) CometSort 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=3] +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_ticket_number#23 ASC NULLS FIRST, ss_item_sk#22 ASC NULLS FIRST] -(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 - -(36) Scan parquet spark_catalog.default.store_returns +(33) 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 -(37) CometFilter +(34) 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)) -(38) CometProject +(35) 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] -(39) ColumnarToRow [codegen id : 8] +(36) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=4] -(40) Exchange +(37) CometSort 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=4] +Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST] -(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 +(38) CometSortMergeJoin +Left 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] +Right output [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] +Arguments: [ss_ticket_number#23, ss_item_sk#22], [sr_ticket_number#36, sr_item_sk#35], LeftOuter -(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 - -(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] +(39) CometProject 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] +Arguments: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41], [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] -(44) Scan parquet spark_catalog.default.web_sales +(40) 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 [] @@ -335,457 +314,434 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(45) CometFilter +(41) 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) -(46) ReusedExchange [Reuses operator id: 6] +(42) 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) CometBroadcastHashJoin +(43) 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) CometProject +(44) 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: 11] +(45) ReusedExchange [Reuses operator id: 11] Output [2]: [d_date_sk#53, d_year#54] -(50) CometBroadcastHashJoin +(46) 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) CometProject +(47) 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) 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] - -(53) Exchange +(48) CometColumnarExchange 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] +Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(54) Sort [codegen id : 12] +(49) CometSort 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 +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_order_number#43 ASC NULLS FIRST, ws_item_sk#42 ASC NULLS FIRST] -(55) Scan parquet spark_catalog.default.web_returns +(50) 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 -(56) CometFilter +(51) 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)) -(57) CometProject +(52) 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] -(58) ColumnarToRow [codegen id : 13] +(53) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=6] -(59) Exchange +(54) CometSort 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=6] +Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST] -(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 - -(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 +(55) CometSortMergeJoin +Left 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] +Right output [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] +Arguments: [ws_order_number#43, ws_item_sk#42], [wr_order_number#56, wr_item_sk#55], LeftOuter -(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] +(56) CometProject 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] +Arguments: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61], [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] -(63) Union +(57) CometUnion +Child 0 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] +Child 1 Input [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41] +Child 2 Input [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61] -(64) HashAggregate [codegen id : 16] +(58) CometHashAggregate 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] -(65) Exchange +(59) CometColumnarExchange 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=7] +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, CometColumnarShuffle, [plan_id=7] -(66) HashAggregate [codegen id : 17] +(60) CometHashAggregate 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] -(67) HashAggregate [codegen id : 17] +(61) CometHashAggregate 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] -(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=8] +(62) CometColumnarExchange +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] +Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(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] +(63) CometHashAggregate +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] 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] -(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) +(64) CometFilter +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] +Condition : isnotnull(sales_cnt#64) + +(65) CometColumnarExchange +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(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=9] +(66) CometSort +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] +Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65], [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] -(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 +(67) ColumnarToRow [codegen id : 1] +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -(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] +(68) Scan parquet spark_catalog.default.catalog_sales +Output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#74), dynamicpruningexpression(cs_sold_date_sk#74 IN dynamicpruning#75)] +PartitionFilters: [isnotnull(cs_sold_date_sk#70), dynamicpruningexpression(cs_sold_date_sk#70 IN dynamicpruning#71)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(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) +(69) CometFilter +Input [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] +Condition : isnotnull(cs_item_sk#66) -(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] +(70) ReusedExchange [Reuses operator id: 6] +Output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -(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 +(71) CometBroadcastHashJoin +Left output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] +Right output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] +Arguments: [cs_item_sk#66], [i_item_sk#72], Inner, BuildRight -(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] +(72) CometProject +Input [10]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] +Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -(78) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#81, d_year#82] +(73) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#77, d_year#78] 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) 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) CometBroadcastExchange -Input [2]: [d_date_sk#81, d_year#82] -Arguments: [d_date_sk#81, d_year#82] +(74) CometFilter +Input [2]: [d_date_sk#77, d_year#78] +Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) -(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 +(75) CometBroadcastExchange +Input [2]: [d_date_sk#77, d_year#78] +Arguments: [d_date_sk#77, d_year#78] -(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] +(76) CometBroadcastHashJoin +Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] +Right output [2]: [d_date_sk#77, d_year#78] +Arguments: [cs_sold_date_sk#70], [d_date_sk#77], Inner, BuildRight -(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] +(77) CometProject +Input [11]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_date_sk#77, d_year#78] +Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -(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=10] +(78) CometColumnarExchange +Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] +Arguments: hashpartitioning(cs_order_number#67, cs_item_sk#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(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 +(79) CometSort +Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] +Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_order_number#67 ASC NULLS FIRST, cs_item_sk#66 ASC NULLS FIRST] -(86) ReusedExchange [Reuses operator id: 21] -Output [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] +(80) ReusedExchange [Reuses operator id: 19] +Output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -(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 +(81) CometSort +Input [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] +Arguments: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82], [cr_order_number#80 ASC NULLS FIRST, cr_item_sk#79 ASC NULLS FIRST] -(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 +(82) CometSortMergeJoin +Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] +Right output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] +Arguments: [cs_order_number#67, cs_item_sk#66], [cr_order_number#80, cr_item_sk#79], LeftOuter -(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] +(83) CometProject +Input [13]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78, cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] +Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21], [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, (cs_quantity#68 - coalesce(cr_return_quantity#81, 0)) AS sales_cnt#20, (cs_ext_sales_price#69 - coalesce(cr_return_amount#82, 0.00)) AS sales_amt#21] -(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] +(84) Scan parquet spark_catalog.default.store_sales +Output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#91), dynamicpruningexpression(ss_sold_date_sk#91 IN dynamicpruning#92)] +PartitionFilters: [isnotnull(ss_sold_date_sk#87), dynamicpruningexpression(ss_sold_date_sk#87 IN dynamicpruning#88)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(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) - -(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] +(85) CometFilter +Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] +Condition : isnotnull(ss_item_sk#83) -(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 +(86) ReusedExchange [Reuses operator id: 6] +Output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -(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] +(87) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] +Right output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] +Arguments: [ss_item_sk#83], [i_item_sk#89], Inner, BuildRight -(95) ReusedExchange [Reuses operator id: 80] -Output [2]: [d_date_sk#98, d_year#99] +(88) CometProject +Input [10]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] +Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93], [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -(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 +(89) ReusedExchange [Reuses operator id: 75] +Output [2]: [d_date_sk#94, d_year#95] -(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] +(90) CometBroadcastHashJoin +Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] +Right output [2]: [d_date_sk#94, d_year#95] +Arguments: [ss_sold_date_sk#87], [d_date_sk#94], Inner, BuildRight -(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] +(91) CometProject +Input [11]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_date_sk#94, d_year#95] +Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -(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] +(92) CometColumnarExchange +Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] +Arguments: hashpartitioning(ss_ticket_number#84, ss_item_sk#83, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [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 +(93) CometSort +Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] +Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_ticket_number#84 ASC NULLS FIRST, ss_item_sk#83 ASC NULLS FIRST] -(101) ReusedExchange [Reuses operator id: 40] -Output [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] +(94) ReusedExchange [Reuses operator id: 36] +Output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -(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 +(95) CometSort +Input [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] +Arguments: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99], [sr_ticket_number#97 ASC NULLS FIRST, sr_item_sk#96 ASC NULLS FIRST] -(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 +(96) CometSortMergeJoin +Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] +Right output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] +Arguments: [ss_ticket_number#84, ss_item_sk#83], [sr_ticket_number#97, sr_item_sk#96], LeftOuter -(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] +(97) CometProject +Input [13]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95, sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] +Arguments: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41], [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, (ss_quantity#85 - coalesce(sr_return_quantity#98, 0)) AS sales_cnt#40, (ss_ext_sales_price#86 - coalesce(sr_return_amt#99, 0.00)) AS sales_amt#41] -(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] +(98) Scan parquet spark_catalog.default.web_sales +Output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#108), dynamicpruningexpression(ws_sold_date_sk#108 IN dynamicpruning#109)] +PartitionFilters: [isnotnull(ws_sold_date_sk#104), dynamicpruningexpression(ws_sold_date_sk#104 IN dynamicpruning#105)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(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) - -(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] +(99) CometFilter +Input [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] +Condition : isnotnull(ws_item_sk#100) -(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 +(100) ReusedExchange [Reuses operator id: 6] +Output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -(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] +(101) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] +Right output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] +Arguments: [ws_item_sk#100], [i_item_sk#106], Inner, BuildRight -(110) ReusedExchange [Reuses operator id: 80] -Output [2]: [d_date_sk#115, d_year#116] +(102) CometProject +Input [10]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] +Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110], [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -(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 +(103) ReusedExchange [Reuses operator id: 75] +Output [2]: [d_date_sk#111, d_year#112] -(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] +(104) CometBroadcastHashJoin +Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] +Right output [2]: [d_date_sk#111, d_year#112] +Arguments: [ws_sold_date_sk#104], [d_date_sk#111], Inner, BuildRight -(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] +(105) CometProject +Input [11]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_date_sk#111, d_year#112] +Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -(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=12] +(106) CometColumnarExchange +Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] +Arguments: hashpartitioning(ws_order_number#101, ws_item_sk#100, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(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 +(107) CometSort +Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] +Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_order_number#101 ASC NULLS FIRST, ws_item_sk#100 ASC NULLS FIRST] -(116) ReusedExchange [Reuses operator id: 59] -Output [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] +(108) ReusedExchange [Reuses operator id: 53] +Output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -(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 +(109) CometSort +Input [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] +Arguments: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116], [wr_order_number#114 ASC NULLS FIRST, wr_item_sk#113 ASC NULLS FIRST] -(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 +(110) CometSortMergeJoin +Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] +Right output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] +Arguments: [ws_order_number#101, ws_item_sk#100], [wr_order_number#114, wr_item_sk#113], LeftOuter -(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] +(111) CometProject +Input [13]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112, wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] +Arguments: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61], [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, (ws_quantity#102 - coalesce(wr_return_quantity#115, 0)) AS sales_cnt#60, (ws_ext_sales_price#103 - coalesce(wr_return_amt#116, 0.00)) AS sales_amt#61] -(120) Union +(112) CometUnion +Child 0 Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] +Child 1 Input [7]: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41] +Child 2 Input [7]: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61] -(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] +(113) CometHashAggregate +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] +Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 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] -(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=13] +(114) CometColumnarExchange +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] +Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(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] +(115) CometHashAggregate +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] +Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 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] -(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] +(116) CometHashAggregate +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] +Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] 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] -(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=14] +(117) CometColumnarExchange +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] +Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] -(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] +(118) CometHashAggregate +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] +Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] 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] -(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) +(119) CometFilter +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] +Condition : isnotnull(sales_cnt#118) + +(120) CometColumnarExchange +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] +Arguments: hashpartitioning(i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] -(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=15] +(121) CometSort +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] +Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119], [i_brand_id#73 ASC NULLS FIRST, i_class_id#74 ASC NULLS FIRST, i_category_id#75 ASC NULLS FIRST, i_manufact_id#76 ASC NULLS FIRST] -(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 +(122) ColumnarToRow [codegen id : 2] +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -(130) SortMergeJoin [codegen id : 39] +(123) SortMergeJoin [codegen id : 3] 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] +Right keys [4]: [i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] Join type: Inner -Join condition: ((cast(sales_cnt#68 as decimal(17,2)) / cast(sales_cnt#123 as decimal(17,2))) < 0.90000000000000000000) +Join condition: ((cast(sales_cnt#64 as decimal(17,2)) / cast(sales_cnt#118 as decimal(17,2))) < 0.90000000000000000000) -(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] +(124) Project [codegen id : 3] +Output [10]: [d_year#78 AS prev_year#120, d_year#14 AS year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#118 AS prev_yr_cnt#122, sales_cnt#64 AS curr_yr_cnt#123, (sales_cnt#64 - sales_cnt#118) AS sales_cnt_diff#124, (sales_amt#65 - sales_amt#119) AS sales_amt_diff#125] +Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65, d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -(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] +(125) TakeOrderedAndProject +Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] +Arguments: 100, [sales_cnt_diff#124 ASC NULLS FIRST], [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (136) -+- * ColumnarToRow (135) - +- CometFilter (134) - +- CometScan parquet spark_catalog.default.date_dim (133) +BroadcastExchange (129) ++- * ColumnarToRow (128) + +- CometFilter (127) + +- CometScan parquet spark_catalog.default.date_dim (126) -(133) Scan parquet spark_catalog.default.date_dim +(126) 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 -(134) CometFilter +(127) 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)) -(135) ColumnarToRow [codegen id : 1] +(128) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] -(136) BroadcastExchange +(129) BroadcastExchange Input [2]: [d_date_sk#13, d_year#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 -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) +Subquery:4 Hosting operator id = 68 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 +BroadcastExchange (133) ++- * ColumnarToRow (132) + +- CometFilter (131) + +- CometScan parquet spark_catalog.default.date_dim (130) -(137) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#81, d_year#82] +(130) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#77, d_year#78] 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 -(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)) +(131) CometFilter +Input [2]: [d_date_sk#77, d_year#78] +Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) -(139) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#81, d_year#82] +(132) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#77, d_year#78] -(140) BroadcastExchange -Input [2]: [d_date_sk#81, d_year#82] +(133) BroadcastExchange +Input [2]: [d_date_sk#77, d_year#78] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] -Subquery:5 Hosting operator id = 90 Hosting Expression = ss_sold_date_sk#91 IN dynamicpruning#75 +Subquery:5 Hosting operator id = 84 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#71 -Subquery:6 Hosting operator id = 105 Hosting Expression = ws_sold_date_sk#108 IN dynamicpruning#75 +Subquery:6 Hosting operator id = 98 Hosting Expression = ws_sold_date_sk#104 IN dynamicpruning#71 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/simplified.txt index 27a4dcb67..a942e8071 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/simplified.txt @@ -1,229 +1,150 @@ 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 (39) + WholeStageCodegen (3) 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 (19) - Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - 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 (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 (16) - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - InputAdapter - Union - 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 (2) - Sort [cs_order_number,cs_item_sk] - InputAdapter - Exchange [cs_order_number,cs_item_sk] #4 - 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_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,d_date_sk,d_year] - 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,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_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 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,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 [d_date_sk,d_year] #7 - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - WholeStageCodegen (4) - Sort [cr_order_number,cr_item_sk] - InputAdapter - 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_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_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 (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 (7) - Sort [ss_ticket_number,ss_item_sk] - 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_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,d_date_sk,d_year] - 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,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_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 - 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 (9) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - 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_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_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 (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 (12) - Sort [ws_order_number,ws_item_sk] - 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_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,d_date_sk,d_year] - 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,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_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 - 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 (14) - Sort [wr_order_number,wr_item_sk] - InputAdapter - 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_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 + CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum,sum,sum(sales_cnt),sum(UnscaledValue(sales_amt))] + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum,sales_cnt,sales_amt] + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometSort [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] + CometColumnarExchange [cs_order_number,cs_item_sk] #4 + 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_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,d_date_sk,d_year] + 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,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_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 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,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 [d_date_sk,d_year] #7 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometColumnarExchange [cr_order_number,cr_item_sk] #8 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometSort [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] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #9 + 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_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,d_date_sk,d_year] + 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,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_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 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #7 + CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometColumnarExchange [sr_ticket_number,sr_item_sk] #10 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometSort [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] + CometColumnarExchange [ws_order_number,ws_item_sk] #11 + 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_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,d_date_sk,d_year] + 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,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_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 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #7 + CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometColumnarExchange [wr_order_number,wr_item_sk] #12 + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_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 (38) - Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] + WholeStageCodegen (2) + ColumnarToRow InputAdapter - 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] #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] #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 (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 (21) - Sort [cs_order_number,cs_item_sk] - 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_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,d_date_sk,d_year] - 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,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_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 #17 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometBroadcastExchange [d_date_sk,d_year] #18 - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - WholeStageCodegen (23) - Sort [cr_order_number,cr_item_sk] - InputAdapter - 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 (26) - Sort [ss_ticket_number,ss_item_sk] - 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_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,d_date_sk,d_year] - 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,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_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 - 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 (28) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - 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 (31) - Sort [ws_order_number,ws_item_sk] - 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_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,d_date_sk,d_year] - 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,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_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 - 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 (33) - Sort [wr_order_number,wr_item_sk] - InputAdapter - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #12 + CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 + CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum,sum,sum(sales_cnt),sum(UnscaledValue(sales_amt))] + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum,sales_cnt,sales_amt] + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometSort [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] + CometColumnarExchange [cs_order_number,cs_item_sk] #16 + 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_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,d_date_sk,d_year] + 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,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_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 #17 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + CometBroadcastExchange [d_date_sk,d_year] #18 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #8 + CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometSort [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] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #19 + 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_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,d_date_sk,d_year] + 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,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_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 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #18 + CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #10 + CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometSort [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] + CometColumnarExchange [ws_order_number,ws_item_sk] #20 + 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_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,d_date_sk,d_year] + 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,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_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 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #18 + CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + 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-spark3_5/q76/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/explain.txt index 27a4a8c3f..d1e7938f7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (34) -+- * HashAggregate (33) - +- Exchange (32) - +- * ColumnarToRow (31) +* ColumnarToRow (34) ++- CometTakeOrderedAndProject (33) + +- CometHashAggregate (32) + +- CometColumnarExchange (31) +- CometHashAggregate (30) +- CometUnion (29) :- CometProject (12) @@ -177,21 +177,19 @@ Input [6]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, ext_sales_ 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))] -(31) ColumnarToRow [codegen id : 1] +(31) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(33) HashAggregate [codegen id : 2] +(32) CometHashAggregate 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)#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] -(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] +(33) CometTakeOrderedAndProject +Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#39, sales_amt#40] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[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], output=[channel#10,col_name#11,d_year#8,d_qoy#9,i_category#6,sales_cnt#39,sales_amt#40]), [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#39, sales_amt#40], 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#39, sales_amt#40] + +(34) ColumnarToRow [codegen id : 1] +Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#39, sales_amt#40] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/simplified.txt index e463296d4..a3a571fa1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/simplified.txt @@ -1,38 +1,36 @@ -TakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [channel,col_name,d_year,d_qoy,i_category,count,sum,ext_sales_price] - CometUnion [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometProject [ss_store_sk,ss_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] - CometProject [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_category] - CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_category] #2 - CometFilter [i_item_sk,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_category] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #3 - CometFilter [d_date_sk,d_year,d_qoy] - 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_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] - CometProject [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_category] - CometFilter [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_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_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] - CometProject [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category] - CometBroadcastHashJoin [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_category] - CometFilter [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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 +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] + CometHashAggregate [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt,count,sum,count(1),sum(UnscaledValue(ext_sales_price))] + CometColumnarExchange [channel,col_name,d_year,d_qoy,i_category] #1 + CometHashAggregate [channel,col_name,d_year,d_qoy,i_category,count,sum,ext_sales_price] + CometUnion [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] + CometProject [ss_store_sk,ss_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] + CometProject [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_category] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange [i_item_sk,i_category] #2 + CometFilter [i_item_sk,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_category] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #3 + CometFilter [d_date_sk,d_year,d_qoy] + 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_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] + CometProject [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_category] + CometFilter [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_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_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] + CometProject [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category] + CometBroadcastHashJoin [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_category] + CometFilter [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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-spark3_5/q77/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/explain.txt index a61e0be7d..2c2016542 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/explain.txt @@ -1,90 +1,90 @@ == Physical Plan == 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) + +- * ColumnarToRow (84) + +- CometColumnarExchange (83) + +- RowToColumnar (82) + +- * HashAggregate (81) + +- * Expand (80) + +- Union (79) + :- * ColumnarToRow (31) + : +- CometProject (30) + : +- CometBroadcastHashJoin (29) + : :- CometHashAggregate (16) + : : +- CometColumnarExchange (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) + : +- CometBroadcastExchange (28) + : +- CometHashAggregate (27) + : +- CometColumnarExchange (26) + : +- CometHashAggregate (25) + : +- CometProject (24) + : +- CometBroadcastHashJoin (23) + : :- CometProject (21) + : : +- CometBroadcastHashJoin (20) + : : :- CometFilter (18) + : : : +- CometScan parquet spark_catalog.default.store_returns (17) + : : +- ReusedExchange (19) + : +- ReusedExchange (22) + :- * Project (50) + : +- * BroadcastNestedLoopJoin Inner BuildLeft (49) + : :- BroadcastExchange (40) + : : +- * ColumnarToRow (39) + : : +- CometHashAggregate (38) + : : +- CometColumnarExchange (37) + : : +- CometHashAggregate (36) + : : +- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometScan parquet spark_catalog.default.catalog_sales (32) + : : +- ReusedExchange (33) + : +- * ColumnarToRow (48) + : +- CometHashAggregate (47) + : +- CometColumnarExchange (46) + : +- CometHashAggregate (45) + : +- CometProject (44) + : +- CometBroadcastHashJoin (43) + : :- CometScan parquet spark_catalog.default.catalog_returns (41) + : +- ReusedExchange (42) + +- * ColumnarToRow (78) + +- CometProject (77) + +- CometBroadcastHashJoin (76) + :- CometHashAggregate (63) + : +- CometColumnarExchange (62) + : +- CometHashAggregate (61) + : +- CometProject (60) + : +- CometBroadcastHashJoin (59) + : :- CometProject (55) + : : +- CometBroadcastHashJoin (54) + : : :- CometFilter (52) + : : : +- CometScan parquet spark_catalog.default.web_sales (51) + : : +- ReusedExchange (53) + : +- CometBroadcastExchange (58) + : +- CometFilter (57) + : +- CometScan parquet spark_catalog.default.web_page (56) + +- CometBroadcastExchange (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) + +- CometColumnarExchange (73) + +- CometHashAggregate (72) + +- CometProject (71) + +- CometBroadcastHashJoin (70) + :- CometProject (68) + : +- CometBroadcastHashJoin (67) + : :- CometFilter (65) + : : +- CometScan parquet spark_catalog.default.web_returns (64) + : +- ReusedExchange (66) + +- ReusedExchange (69) (1) Scan parquet spark_catalog.default.store_sales @@ -156,340 +156,326 @@ 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))] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarExchange Input [3]: [s_store_sk#8, sum#9, sum#10] +Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(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] +(16) CometHashAggregate 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))#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] -(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] +(17) Scan parquet spark_catalog.default.store_returns +Output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#18), dynamicpruningexpression(sr_returned_date_sk#18 IN dynamicpruning#19)] +PartitionFilters: [isnotnull(sr_returned_date_sk#14), dynamicpruningexpression(sr_returned_date_sk#14 IN dynamicpruning#15)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(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) - -(20) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#20] - -(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 - -(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] - -(23) ReusedExchange [Reuses operator id: 11] -Output [1]: [s_store_sk#21] - -(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 - -(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] - -(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 +(18) CometFilter +Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] +Condition : isnotnull(sr_store_sk#11) + +(19) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#16] + +(20) CometBroadcastHashJoin +Left output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] +Right output [1]: [d_date_sk#16] +Arguments: [sr_returned_date_sk#14], [d_date_sk#16], Inner, BuildRight + +(21) CometProject +Input [5]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14, d_date_sk#16] +Arguments: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13], [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] + +(22) ReusedExchange [Reuses operator id: 11] +Output [1]: [s_store_sk#17] + +(23) CometBroadcastHashJoin +Left output [3]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] +Right output [1]: [s_store_sk#17] +Arguments: [sr_store_sk#11], [s_store_sk#17], Inner, BuildRight + +(24) CometProject +Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, s_store_sk#17] +Arguments: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17], [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] + +(25) CometHashAggregate +Input [3]: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] +Keys [1]: [s_store_sk#17] +Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#12)), partial_sum(UnscaledValue(sr_net_loss#13))] + +(26) CometColumnarExchange +Input [3]: [s_store_sk#17, sum#18, sum#19] +Arguments: hashpartitioning(s_store_sk#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(27) CometHashAggregate +Input [3]: [s_store_sk#17, sum#18, sum#19] +Keys [1]: [s_store_sk#17] +Functions [2]: [sum(UnscaledValue(sr_return_amt#12)), sum(UnscaledValue(sr_net_loss#13))] + +(28) CometBroadcastExchange +Input [3]: [s_store_sk#17, returns#20, profit_loss#21] +Arguments: [s_store_sk#17, returns#20, profit_loss#21] + +(29) CometBroadcastHashJoin +Left output [3]: [s_store_sk#8, sales#22, profit#23] +Right output [3]: [s_store_sk#17, returns#20, profit_loss#21] +Arguments: [s_store_sk#8], [s_store_sk#17], LeftOuter, BuildRight -(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] +(30) CometProject +Input [6]: [s_store_sk#8, sales#22, profit#23, s_store_sk#17, returns#20, profit_loss#21] +Arguments: [sales#22, returns#24, profit#25, channel#26, id#27], [sales#22, coalesce(returns#20, 0.00) AS returns#24, (profit#23 - coalesce(profit_loss#21, 0.00)) AS profit#25, store channel AS channel#26, s_store_sk#8 AS id#27] -(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] +(31) ColumnarToRow [codegen id : 1] +Input [5]: [sales#22, returns#24, profit#25, channel#26, id#27] + +(32) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#35), dynamicpruningexpression(cs_sold_date_sk#35 IN dynamicpruning#36)] +PartitionFilters: [isnotnull(cs_sold_date_sk#31), dynamicpruningexpression(cs_sold_date_sk#31 IN dynamicpruning#32)] ReadSchema: struct -(34) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#37] +(33) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#33] -(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) CometBroadcastHashJoin +Left output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] +Right output [1]: [d_date_sk#33] +Arguments: [cs_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight -(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) CometProject +Input [5]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31, d_date_sk#33] +Arguments: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30], [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] -(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))] +(36) CometHashAggregate +Input [3]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] +Keys [1]: [cs_call_center_sk#28] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#29)), partial_sum(UnscaledValue(cs_net_profit#30))] -(38) ColumnarToRow [codegen id : 5] -Input [3]: [cs_call_center_sk#32, sum#38, sum#39] +(37) CometColumnarExchange +Input [3]: [cs_call_center_sk#28, sum#34, sum#35] +Arguments: hashpartitioning(cs_call_center_sk#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(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] +(38) CometHashAggregate +Input [3]: [cs_call_center_sk#28, sum#34, sum#35] +Keys [1]: [cs_call_center_sk#28] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#29)), sum(UnscaledValue(cs_net_profit#30))] -(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] +(39) ColumnarToRow [codegen id : 2] +Input [3]: [cs_call_center_sk#28, sales#36, profit#37] -(41) BroadcastExchange -Input [3]: [cs_call_center_sk#32, sales#42, profit#43] -Arguments: IdentityBroadcastMode, [plan_id=5] +(40) BroadcastExchange +Input [3]: [cs_call_center_sk#28, sales#36, profit#37] +Arguments: IdentityBroadcastMode, [plan_id=4] -(42) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#44, cr_net_loss#45, cr_returned_date_sk#46] +(41) Scan parquet spark_catalog.default.catalog_returns +Output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#46), dynamicpruningexpression(cr_returned_date_sk#46 IN dynamicpruning#47)] +PartitionFilters: [isnotnull(cr_returned_date_sk#40), dynamicpruningexpression(cr_returned_date_sk#40 IN dynamicpruning#41)] ReadSchema: struct -(43) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#48] +(42) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#42] -(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) CometBroadcastHashJoin +Left output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] +Right output [1]: [d_date_sk#42] +Arguments: [cr_returned_date_sk#40], [d_date_sk#42], Inner, BuildRight -(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) CometProject +Input [4]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40, d_date_sk#42] +Arguments: [cr_return_amount#38, cr_net_loss#39], [cr_return_amount#38, cr_net_loss#39] -(46) CometHashAggregate -Input [2]: [cr_return_amount#44, cr_net_loss#45] +(45) CometHashAggregate +Input [2]: [cr_return_amount#38, cr_net_loss#39] Keys: [] -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] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#38)), partial_sum(UnscaledValue(cr_net_loss#39))] -(48) Exchange -Input [2]: [sum#49, sum#50] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +(46) CometColumnarExchange +Input [2]: [sum#43, sum#44] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(49) HashAggregate -Input [2]: [sum#49, sum#50] +(47) CometHashAggregate +Input [2]: [sum#43, sum#44] Keys: [] -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] +Functions [2]: [sum(UnscaledValue(cr_return_amount#38)), sum(UnscaledValue(cr_net_loss#39))] -(50) BroadcastNestedLoopJoin [codegen id : 8] +(48) ColumnarToRow +Input [2]: [returns#45, profit_loss#46] + +(49) BroadcastNestedLoopJoin [codegen id : 3] Join type: Inner Join condition: None -(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) Project [codegen id : 3] +Output [5]: [sales#36, returns#45, (profit#37 - profit_loss#46) AS profit#47, catalog channel AS channel#48, cs_call_center_sk#28 AS id#49] +Input [5]: [cs_call_center_sk#28, sales#36, profit#37, returns#45, profit_loss#46] -(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] +(51) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#61), dynamicpruningexpression(ws_sold_date_sk#61 IN dynamicpruning#62)] +PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct -(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) +(52) CometFilter +Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] +Condition : isnotnull(ws_web_page_sk#50) -(54) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#63] +(53) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#55] -(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 +(54) CometBroadcastHashJoin +Left output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] +Right output [1]: [d_date_sk#55] +Arguments: [ws_sold_date_sk#53], [d_date_sk#55], Inner, BuildRight -(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] +(55) CometProject +Input [5]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53, d_date_sk#55] +Arguments: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52], [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] -(57) Scan parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#64] +(56) Scan parquet spark_catalog.default.web_page +Output [1]: [wp_web_page_sk#56] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct -(58) CometFilter -Input [1]: [wp_web_page_sk#64] -Condition : isnotnull(wp_web_page_sk#64) - -(59) CometBroadcastExchange -Input [1]: [wp_web_page_sk#64] -Arguments: [wp_web_page_sk#64] +(57) CometFilter +Input [1]: [wp_web_page_sk#56] +Condition : isnotnull(wp_web_page_sk#56) -(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 +(58) CometBroadcastExchange +Input [1]: [wp_web_page_sk#56] +Arguments: [wp_web_page_sk#56] -(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] +(59) CometBroadcastHashJoin +Left output [3]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] +Right output [1]: [wp_web_page_sk#56] +Arguments: [ws_web_page_sk#50], [wp_web_page_sk#56], Inner, BuildRight -(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))] +(60) CometProject +Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] +Arguments: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56], [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] -(63) ColumnarToRow [codegen id : 9] -Input [3]: [wp_web_page_sk#64, sum#65, sum#66] +(61) CometHashAggregate +Input [3]: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] +Keys [1]: [wp_web_page_sk#56] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#51)), partial_sum(UnscaledValue(ws_net_profit#52))] -(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] +(62) CometColumnarExchange +Input [3]: [wp_web_page_sk#56, sum#57, sum#58] +Arguments: hashpartitioning(wp_web_page_sk#56, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(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] +(63) CometHashAggregate +Input [3]: [wp_web_page_sk#56, sum#57, sum#58] +Keys [1]: [wp_web_page_sk#56] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#51)), sum(UnscaledValue(ws_net_profit#52))] -(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] +(64) Scan parquet spark_catalog.default.web_returns +Output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#74), dynamicpruningexpression(wr_returned_date_sk#74 IN dynamicpruning#75)] +PartitionFilters: [isnotnull(wr_returned_date_sk#62), dynamicpruningexpression(wr_returned_date_sk#62 IN dynamicpruning#63)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct -(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) +(65) CometFilter +Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] +Condition : isnotnull(wr_web_page_sk#59) -(68) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#76] +(66) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#64] -(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 +(67) CometBroadcastHashJoin +Left output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] +Right output [1]: [d_date_sk#64] +Arguments: [wr_returned_date_sk#62], [d_date_sk#64], Inner, BuildRight -(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] +(68) CometProject +Input [5]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62, d_date_sk#64] +Arguments: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61], [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] -(71) ReusedExchange [Reuses operator id: 59] -Output [1]: [wp_web_page_sk#77] +(69) ReusedExchange [Reuses operator id: 58] +Output [1]: [wp_web_page_sk#65] -(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 +(70) CometBroadcastHashJoin +Left output [3]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] +Right output [1]: [wp_web_page_sk#65] +Arguments: [wr_web_page_sk#59], [wp_web_page_sk#65], 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] +(71) CometProject +Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] +Arguments: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65], [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] + +(72) CometHashAggregate +Input [3]: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] +Keys [1]: [wp_web_page_sk#65] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#60)), partial_sum(UnscaledValue(wr_net_loss#61))] + +(73) CometColumnarExchange +Input [3]: [wp_web_page_sk#65, sum#66, sum#67] +Arguments: hashpartitioning(wp_web_page_sk#65, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] (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] +Input [3]: [wp_web_page_sk#65, sum#66, sum#67] +Keys [1]: [wp_web_page_sk#65] +Functions [2]: [sum(UnscaledValue(wr_return_amt#60)), sum(UnscaledValue(wr_net_loss#61))] -(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 +(75) CometBroadcastExchange +Input [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] +Arguments: [wp_web_page_sk#65, returns#68, profit_loss#69] + +(76) CometBroadcastHashJoin +Left output [3]: [wp_web_page_sk#56, sales#70, profit#71] +Right output [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] +Arguments: [wp_web_page_sk#56], [wp_web_page_sk#65], LeftOuter, BuildRight + +(77) CometProject +Input [6]: [wp_web_page_sk#56, sales#70, profit#71, wp_web_page_sk#65, returns#68, profit_loss#69] +Arguments: [sales#70, returns#72, profit#73, channel#74, id#75], [sales#70, coalesce(returns#68, 0.00) AS returns#72, (profit#71 - coalesce(profit_loss#69, 0.00)) AS profit#73, web channel AS channel#74, wp_web_page_sk#56 AS id#75] -(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] +(78) ColumnarToRow [codegen id : 4] +Input [5]: [sales#70, returns#72, profit#73, channel#74, id#75] -(81) Union +(79) Union -(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] +(80) Expand [codegen id : 5] +Input [5]: [sales#22, returns#24, profit#25, channel#26, id#27] +Arguments: [[sales#22, returns#24, profit#25, channel#26, id#27, 0], [sales#22, returns#24, profit#25, channel#26, null, 1], [sales#22, returns#24, profit#25, null, null, 3]], [sales#22, returns#24, profit#25, channel#76, id#77, spark_grouping_id#78] -(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] +(81) HashAggregate [codegen id : 5] +Input [6]: [sales#22, returns#24, profit#25, channel#76, id#77, spark_grouping_id#78] +Keys [3]: [channel#76, id#77, spark_grouping_id#78] +Functions [3]: [partial_sum(sales#22), partial_sum(returns#24), partial_sum(profit#25)] +Aggregate Attributes [6]: [sum#79, isEmpty#80, sum#81, isEmpty#82, sum#83, isEmpty#84] +Results [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] -(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] +(82) RowToColumnar +Input [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] -(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] +(83) CometColumnarExchange +Input [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] +Arguments: hashpartitioning(channel#76, id#77, spark_grouping_id#78, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(84) ColumnarToRow [codegen id : 6] +Input [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] + +(85) HashAggregate [codegen id : 6] +Input [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] +Keys [3]: [channel#76, id#77, spark_grouping_id#78] +Functions [3]: [sum(sales#22), sum(returns#24), sum(profit#25)] +Aggregate Attributes [3]: [sum(sales#22)#91, sum(returns#24)#92, sum(profit#25)#93] +Results [5]: [channel#76, id#77, sum(sales#22)#91 AS sales#94, sum(returns#24)#92 AS returns#95, sum(profit#25)#93 AS profit#96] (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] +Input [5]: [channel#76, id#77, sales#94, returns#95, profit#96] +Arguments: 100, [channel#76 ASC NULLS FIRST, id#77 ASC NULLS FIRST], [channel#76, id#77, sales#94, returns#95, profit#96] ===== Subqueries ===== @@ -521,16 +507,16 @@ Input [1]: [d_date_sk#6] (91) BroadcastExchange Input [1]: [d_date_sk#6] -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=9] -Subquery:2 Hosting operator id = 18 Hosting Expression = sr_returned_date_sk#18 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 17 Hosting Expression = sr_returned_date_sk#14 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#35 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#31 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 42 Hosting Expression = cr_returned_date_sk#46 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 41 Hosting Expression = cr_returned_date_sk#40 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 52 Hosting Expression = ws_sold_date_sk#61 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 51 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 66 Hosting Expression = wr_returned_date_sk#74 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 64 Hosting Expression = wr_returned_date_sk#62 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/simplified.txt index f9088f784..605180016 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/simplified.txt @@ -1,22 +1,22 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (14) + WholeStageCodegen (6) 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) - 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) - 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel,id,spark_grouping_id] #1 + RowToColumnar + WholeStageCodegen (5) + 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 (1) + ColumnarToRow InputAdapter - Exchange [s_store_sk] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + CometProject [returns,profit,profit_loss,s_store_sk] [sales,returns,profit,channel,id] + CometBroadcastHashJoin [s_store_sk,sales,profit,s_store_sk,returns,profit_loss] + CometHashAggregate [s_store_sk,sales,profit,sum,sum,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit))] + CometColumnarExchange [s_store_sk] #2 CometHashAggregate [s_store_sk,sum,sum,ss_ext_sales_price,ss_net_profit] CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] @@ -39,64 +39,52 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometBroadcastExchange [s_store_sk] #5 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] - InputAdapter - 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] #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometHashAggregate [s_store_sk,sum,sum,sr_return_amt,sr_net_loss] - CometProject [sr_return_amt,sr_net_loss,s_store_sk] - CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] - CometProject [sr_store_sk,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_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 - 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 #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] #9 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter + CometBroadcastExchange [s_store_sk,returns,profit_loss] #6 + CometHashAggregate [s_store_sk,returns,profit_loss,sum,sum,sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss))] + CometColumnarExchange [s_store_sk] #7 + CometHashAggregate [s_store_sk,sum,sum,sr_return_amt,sr_net_loss] + CometProject [sr_return_amt,sr_net_loss,s_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] + CometProject [sr_store_sk,sr_return_amt,sr_net_loss] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_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 + ReusedExchange [d_date_sk] #4 + ReusedExchange [s_store_sk] #5 + WholeStageCodegen (3) + Project [sales,returns,profit,profit_loss,cs_call_center_sk] + BroadcastNestedLoopJoin + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometHashAggregate [cs_call_center_sk,sales,profit,sum,sum,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] + CometColumnarExchange [cs_call_center_sk] #9 CometHashAggregate [cs_call_center_sk,sum,sum,cs_ext_sales_price,cs_net_profit] CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,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 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 #10 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometHashAggregate [returns,profit_loss,sum,sum,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] + CometColumnarExchange #10 CometHashAggregate [sum,sum,cr_return_amount,cr_net_loss] CometProject [cr_return_amount,cr_net_loss] CometBroadcastHashJoin [cr_return_amount,cr_net_loss,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 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] + WholeStageCodegen (4) + ColumnarToRow InputAdapter - Exchange [wp_web_page_sk] #11 - WholeStageCodegen (9) - ColumnarToRow - InputAdapter + CometProject [returns,profit,profit_loss,wp_web_page_sk] [sales,returns,profit,channel,id] + CometBroadcastHashJoin [wp_web_page_sk,sales,profit,wp_web_page_sk,returns,profit_loss] + CometHashAggregate [wp_web_page_sk,sales,profit,sum,sum,sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit))] + CometColumnarExchange [wp_web_page_sk] #11 CometHashAggregate [wp_web_page_sk,sum,sum,ws_ext_sales_price,ws_net_profit] CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] @@ -109,22 +97,16 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometBroadcastExchange [wp_web_page_sk] #12 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] - InputAdapter - 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] #14 - WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometHashAggregate [wp_web_page_sk,sum,sum,wr_return_amt,wr_net_loss] - CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] - CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] - CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk] - CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_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 - ReusedExchange [d_date_sk] #4 - ReusedExchange [wp_web_page_sk] #12 + CometBroadcastExchange [wp_web_page_sk,returns,profit_loss] #13 + CometHashAggregate [wp_web_page_sk,returns,profit_loss,sum,sum,sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss))] + CometColumnarExchange [wp_web_page_sk] #14 + CometHashAggregate [wp_web_page_sk,sum,sum,wr_return_amt,wr_net_loss] + CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] + CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk] + CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_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 + 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-spark3_5/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/explain.txt index 7f2688112..2ffeb0737 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/explain.txt @@ -1,74 +1,71 @@ == Physical Plan == -TakeOrderedAndProject (70) -+- * Project (69) - +- * SortMergeJoin Inner (68) - :- * Project (45) - : +- * SortMergeJoin Inner (44) - : :- * Sort (21) - : : +- * HashAggregate (20) - : : +- Exchange (19) - : : +- * HashAggregate (18) - : : +- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Project (14) - : : : +- * Filter (13) - : : : +- * SortMergeJoin LeftOuter (12) - : : : :- * Sort (5) - : : : : +- Exchange (4) - : : : : +- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- * Sort (11) - : : : +- Exchange (10) - : : : +- * ColumnarToRow (9) - : : : +- CometProject (8) - : : : +- CometFilter (7) - : : : +- CometScan parquet spark_catalog.default.store_returns (6) - : : +- ReusedExchange (15) - : +- * Sort (43) - : +- * Filter (42) - : +- * HashAggregate (41) - : +- Exchange (40) - : +- * HashAggregate (39) - : +- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * Project (35) - : : +- * Filter (34) - : : +- * SortMergeJoin LeftOuter (33) - : : :- * Sort (26) - : : : +- Exchange (25) - : : : +- * ColumnarToRow (24) - : : : +- CometFilter (23) - : : : +- CometScan parquet spark_catalog.default.web_sales (22) - : : +- * Sort (32) - : : +- Exchange (31) - : : +- * ColumnarToRow (30) - : : +- CometProject (29) - : : +- CometFilter (28) - : : +- CometScan parquet spark_catalog.default.web_returns (27) - : +- ReusedExchange (36) - +- * Sort (67) - +- * Filter (66) - +- * HashAggregate (65) - +- Exchange (64) - +- * HashAggregate (63) - +- * Project (62) - +- * BroadcastHashJoin Inner BuildRight (61) - :- * Project (59) - : +- * Filter (58) - : +- * SortMergeJoin LeftOuter (57) - : :- * Sort (50) - : : +- Exchange (49) - : : +- * ColumnarToRow (48) - : : +- CometFilter (47) - : : +- CometScan parquet spark_catalog.default.catalog_sales (46) - : +- * Sort (56) - : +- Exchange (55) - : +- * ColumnarToRow (54) - : +- CometProject (53) - : +- CometFilter (52) - : +- CometScan parquet spark_catalog.default.catalog_returns (51) - +- ReusedExchange (60) +TakeOrderedAndProject (67) ++- * Project (66) + +- * ColumnarToRow (65) + +- CometSortMergeJoin (64) + :- CometProject (43) + : +- CometSortMergeJoin (42) + : :- CometSort (21) + : : +- CometHashAggregate (20) + : : +- CometColumnarExchange (19) + : : +- CometHashAggregate (18) + : : +- CometProject (17) + : : +- CometBroadcastHashJoin (16) + : : :- CometProject (12) + : : : +- CometFilter (11) + : : : +- CometSortMergeJoin (10) + : : : :- CometSort (4) + : : : : +- CometColumnarExchange (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometSort (9) + : : : +- CometColumnarExchange (8) + : : : +- CometProject (7) + : : : +- CometFilter (6) + : : : +- CometScan parquet spark_catalog.default.store_returns (5) + : : +- CometBroadcastExchange (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.date_dim (13) + : +- CometSort (41) + : +- CometFilter (40) + : +- CometHashAggregate (39) + : +- CometColumnarExchange (38) + : +- CometHashAggregate (37) + : +- CometProject (36) + : +- CometBroadcastHashJoin (35) + : :- CometProject (33) + : : +- CometFilter (32) + : : +- CometSortMergeJoin (31) + : : :- CometSort (25) + : : : +- CometColumnarExchange (24) + : : : +- CometFilter (23) + : : : +- CometScan parquet spark_catalog.default.web_sales (22) + : : +- CometSort (30) + : : +- CometColumnarExchange (29) + : : +- CometProject (28) + : : +- CometFilter (27) + : : +- CometScan parquet spark_catalog.default.web_returns (26) + : +- ReusedExchange (34) + +- CometSort (63) + +- CometFilter (62) + +- CometHashAggregate (61) + +- CometColumnarExchange (60) + +- CometHashAggregate (59) + +- CometProject (58) + +- CometBroadcastHashJoin (57) + :- CometProject (55) + : +- CometFilter (54) + : +- CometSortMergeJoin (53) + : :- CometSort (47) + : : +- CometColumnarExchange (46) + : : +- CometFilter (45) + : : +- CometScan parquet spark_catalog.default.catalog_sales (44) + : +- CometSort (52) + : +- CometColumnarExchange (51) + : +- CometProject (50) + : +- CometFilter (49) + : +- CometScan parquet spark_catalog.default.catalog_returns (48) + +- ReusedExchange (56) (1) Scan parquet spark_catalog.default.store_sales @@ -83,349 +80,326 @@ ReadSchema: struct -(7) CometFilter +(6) CometFilter Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] Condition : (isnotnull(sr_ticket_number#10) AND isnotnull(sr_item_sk#9)) -(8) CometProject +(7) CometProject Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_item_sk#9, sr_ticket_number#10] -(9) ColumnarToRow [codegen id : 3] -Input [2]: [sr_item_sk#9, sr_ticket_number#10] - -(10) Exchange +(8) CometColumnarExchange Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(11) Sort [codegen id : 4] +(9) CometSort Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST], false, 0 +Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST] -(12) SortMergeJoin [codegen id : 6] -Left keys [2]: [ss_ticket_number#3, ss_item_sk#1] -Right keys [2]: [sr_ticket_number#10, sr_item_sk#9] -Join type: LeftOuter -Join condition: None +(10) CometSortMergeJoin +Left output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] +Right output [2]: [sr_item_sk#9, sr_ticket_number#10] +Arguments: [ss_ticket_number#3, ss_item_sk#1], [sr_ticket_number#10, sr_item_sk#9], LeftOuter -(13) Filter [codegen id : 6] +(11) CometFilter Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] Condition : isnull(sr_ticket_number#10) -(14) Project [codegen id : 6] -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] +(12) CometProject Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -(15) ReusedExchange [Reuses operator id: 74] +(13) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#12, d_year#13] +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 + +(14) CometFilter +Input [2]: [d_date_sk#12, d_year#13] +Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) + +(15) CometBroadcastExchange +Input [2]: [d_date_sk#12, d_year#13] +Arguments: [d_date_sk#12, d_year#13] -(16) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] +Right output [2]: [d_date_sk#12, d_year#13] +Arguments: [ss_sold_date_sk#7], [d_date_sk#12], Inner, BuildRight -(17) Project [codegen id : 6] -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] +(17) CometProject Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] -(18) HashAggregate [codegen id : 6] +(18) CometHashAggregate Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesale_cost#5)), partial_sum(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [3]: [sum#14, sum#15, sum#16] -Results [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] -(19) Exchange -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] -Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(19) CometColumnarExchange +Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] +Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(20) HashAggregate [codegen id : 7] -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] +(20) CometHashAggregate +Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [3]: [sum(ss_quantity#4)#20, sum(UnscaledValue(ss_wholesale_cost#5))#21, sum(UnscaledValue(ss_sales_price#6))#22] -Results [6]: [d_year#13 AS ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, sum(ss_quantity#4)#20 AS ss_qty#24, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#5))#21,17,2) AS ss_wc#25, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#22,17,2) AS ss_sp#26] -(21) Sort [codegen id : 7] -Input [6]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] -Arguments: [ss_sold_year#23 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], false, 0 +(21) CometSort +Input [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] +Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20], [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST] (22) Scan parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] 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#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#28)] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (23) CometFilter -Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Condition : (isnotnull(ws_item_sk#27) AND isnotnull(ws_bill_customer_sk#28)) +Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] +Condition : (isnotnull(ws_item_sk#21) AND isnotnull(ws_bill_customer_sk#22)) -(24) ColumnarToRow [codegen id : 8] -Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +(24) CometColumnarExchange +Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] +Arguments: hashpartitioning(ws_order_number#23, ws_item_sk#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(25) Exchange -Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Arguments: hashpartitioning(ws_order_number#29, ws_item_sk#27, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(25) CometSort +Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] +Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_order_number#23 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST] -(26) Sort [codegen id : 9] -Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Arguments: [ws_order_number#29 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST], false, 0 - -(27) Scan parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] +(26) Scan parquet spark_catalog.default.web_returns +Output [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct -(28) CometFilter -Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] -Condition : (isnotnull(wr_order_number#36) AND isnotnull(wr_item_sk#35)) - -(29) CometProject -Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] -Arguments: [wr_item_sk#35, wr_order_number#36], [wr_item_sk#35, wr_order_number#36] - -(30) ColumnarToRow [codegen id : 10] -Input [2]: [wr_item_sk#35, wr_order_number#36] - -(31) Exchange -Input [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: hashpartitioning(wr_order_number#36, wr_item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(32) Sort [codegen id : 11] -Input [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: [wr_order_number#36 ASC NULLS FIRST, wr_item_sk#35 ASC NULLS FIRST], false, 0 - -(33) SortMergeJoin [codegen id : 13] -Left keys [2]: [ws_order_number#29, ws_item_sk#27] -Right keys [2]: [wr_order_number#36, wr_item_sk#35] -Join type: LeftOuter -Join condition: None - -(34) Filter [codegen id : 13] -Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] -Condition : isnull(wr_order_number#36) - -(35) Project [codegen id : 13] -Output [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] - -(36) ReusedExchange [Reuses operator id: 74] -Output [2]: [d_date_sk#38, d_year#39] - -(37) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#38] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 13] -Output [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] -Input [8]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, d_date_sk#38, d_year#39] - -(39) HashAggregate [codegen id : 13] -Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] -Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] -Functions [3]: [partial_sum(ws_quantity#30), partial_sum(UnscaledValue(ws_wholesale_cost#31)), partial_sum(UnscaledValue(ws_sales_price#32))] -Aggregate Attributes [3]: [sum#40, sum#41, sum#42] -Results [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] - -(40) Exchange -Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] -Arguments: hashpartitioning(d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, 5), ENSURE_REQUIREMENTS, [plan_id=6] - -(41) HashAggregate [codegen id : 14] -Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] -Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] -Functions [3]: [sum(ws_quantity#30), sum(UnscaledValue(ws_wholesale_cost#31)), sum(UnscaledValue(ws_sales_price#32))] -Aggregate Attributes [3]: [sum(ws_quantity#30)#46, sum(UnscaledValue(ws_wholesale_cost#31))#47, sum(UnscaledValue(ws_sales_price#32))#48] -Results [6]: [d_year#39 AS ws_sold_year#49, ws_item_sk#27, ws_bill_customer_sk#28 AS ws_customer_sk#50, sum(ws_quantity#30)#46 AS ws_qty#51, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#31))#47,17,2) AS ws_wc#52, MakeDecimal(sum(UnscaledValue(ws_sales_price#32))#48,17,2) AS ws_sp#53] - -(42) Filter [codegen id : 14] -Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] -Condition : (coalesce(ws_qty#51, 0) > 0) - -(43) Sort [codegen id : 14] -Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] -Arguments: [ws_sold_year#49 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST, ws_customer_sk#50 ASC NULLS FIRST], false, 0 - -(44) SortMergeJoin [codegen id : 15] -Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50] -Join type: Inner -Join condition: None - -(45) Project [codegen id : 15] -Output [9]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53] -Input [12]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] - -(46) Scan parquet spark_catalog.default.catalog_sales -Output [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +(27) CometFilter +Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] +Condition : (isnotnull(wr_order_number#30) AND isnotnull(wr_item_sk#29)) + +(28) CometProject +Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] +Arguments: [wr_item_sk#29, wr_order_number#30], [wr_item_sk#29, wr_order_number#30] + +(29) CometColumnarExchange +Input [2]: [wr_item_sk#29, wr_order_number#30] +Arguments: hashpartitioning(wr_order_number#30, wr_item_sk#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(30) CometSort +Input [2]: [wr_item_sk#29, wr_order_number#30] +Arguments: [wr_item_sk#29, wr_order_number#30], [wr_order_number#30 ASC NULLS FIRST, wr_item_sk#29 ASC NULLS FIRST] + +(31) CometSortMergeJoin +Left output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] +Right output [2]: [wr_item_sk#29, wr_order_number#30] +Arguments: [ws_order_number#23, ws_item_sk#21], [wr_order_number#30, wr_item_sk#29], LeftOuter + +(32) CometFilter +Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] +Condition : isnull(wr_order_number#30) + +(33) CometProject +Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] +Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] + +(34) ReusedExchange [Reuses operator id: 15] +Output [2]: [d_date_sk#32, d_year#33] + +(35) CometBroadcastHashJoin +Left output [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] +Right output [2]: [d_date_sk#32, d_year#33] +Arguments: [ws_sold_date_sk#27], [d_date_sk#32], Inner, BuildRight + +(36) CometProject +Input [8]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, d_date_sk#32, d_year#33] +Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] + +(37) CometHashAggregate +Input [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] +Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] +Functions [3]: [partial_sum(ws_quantity#24), partial_sum(UnscaledValue(ws_wholesale_cost#25)), partial_sum(UnscaledValue(ws_sales_price#26))] + +(38) CometColumnarExchange +Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] +Arguments: hashpartitioning(d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(39) CometHashAggregate +Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] +Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] +Functions [3]: [sum(ws_quantity#24), sum(UnscaledValue(ws_wholesale_cost#25)), sum(UnscaledValue(ws_sales_price#26))] + +(40) CometFilter +Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] +Condition : (coalesce(ws_qty#39, 0) > 0) + +(41) CometSort +Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] +Arguments: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41], [ws_sold_year#37 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST, ws_customer_sk#38 ASC NULLS FIRST] + +(42) CometSortMergeJoin +Left output [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] +Right output [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] +Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38], Inner + +(43) CometProject +Input [12]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] +Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41], [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] + +(44) Scan parquet spark_catalog.default.catalog_sales +Output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_sold_date_sk#60 IN dynamicpruning#61)] +PartitionFilters: [isnotnull(cs_sold_date_sk#48), dynamicpruningexpression(cs_sold_date_sk#48 IN dynamicpruning#49)] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(47) CometFilter -Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Condition : (isnotnull(cs_item_sk#55) AND isnotnull(cs_bill_customer_sk#54)) - -(48) ColumnarToRow [codegen id : 16] -Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +(45) CometFilter +Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] +Condition : (isnotnull(cs_item_sk#43) AND isnotnull(cs_bill_customer_sk#42)) -(49) Exchange -Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Arguments: hashpartitioning(cs_order_number#56, cs_item_sk#55, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(46) CometColumnarExchange +Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] +Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(50) Sort [codegen id : 17] -Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Arguments: [cs_order_number#56 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST], false, 0 +(47) CometSort +Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] +Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_order_number#44 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST] -(51) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] +(48) Scan parquet spark_catalog.default.catalog_returns +Output [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(52) CometFilter -Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] -Condition : (isnotnull(cr_order_number#63) AND isnotnull(cr_item_sk#62)) - -(53) CometProject -Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] -Arguments: [cr_item_sk#62, cr_order_number#63], [cr_item_sk#62, cr_order_number#63] - -(54) ColumnarToRow [codegen id : 18] -Input [2]: [cr_item_sk#62, cr_order_number#63] - -(55) Exchange -Input [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: hashpartitioning(cr_order_number#63, cr_item_sk#62, 5), ENSURE_REQUIREMENTS, [plan_id=8] - -(56) Sort [codegen id : 19] -Input [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: [cr_order_number#63 ASC NULLS FIRST, cr_item_sk#62 ASC NULLS FIRST], false, 0 - -(57) SortMergeJoin [codegen id : 21] -Left keys [2]: [cs_order_number#56, cs_item_sk#55] -Right keys [2]: [cr_order_number#63, cr_item_sk#62] -Join type: LeftOuter -Join condition: None - -(58) Filter [codegen id : 21] -Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] -Condition : isnull(cr_order_number#63) - -(59) Project [codegen id : 21] -Output [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] - -(60) ReusedExchange [Reuses operator id: 74] -Output [2]: [d_date_sk#65, d_year#66] - -(61) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [cs_sold_date_sk#60] -Right keys [1]: [d_date_sk#65] -Join type: Inner -Join condition: None - -(62) Project [codegen id : 21] -Output [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] -Input [8]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, d_date_sk#65, d_year#66] - -(63) HashAggregate [codegen id : 21] -Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] -Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] -Functions [3]: [partial_sum(cs_quantity#57), partial_sum(UnscaledValue(cs_wholesale_cost#58)), partial_sum(UnscaledValue(cs_sales_price#59))] -Aggregate Attributes [3]: [sum#67, sum#68, sum#69] -Results [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] - -(64) Exchange -Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] -Arguments: hashpartitioning(d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, 5), ENSURE_REQUIREMENTS, [plan_id=9] - -(65) HashAggregate [codegen id : 22] -Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] -Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] -Functions [3]: [sum(cs_quantity#57), sum(UnscaledValue(cs_wholesale_cost#58)), sum(UnscaledValue(cs_sales_price#59))] -Aggregate Attributes [3]: [sum(cs_quantity#57)#73, sum(UnscaledValue(cs_wholesale_cost#58))#74, sum(UnscaledValue(cs_sales_price#59))#75] -Results [6]: [d_year#66 AS cs_sold_year#76, cs_item_sk#55, cs_bill_customer_sk#54 AS cs_customer_sk#77, sum(cs_quantity#57)#73 AS cs_qty#78, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#58))#74,17,2) AS cs_wc#79, MakeDecimal(sum(UnscaledValue(cs_sales_price#59))#75,17,2) AS cs_sp#80] - -(66) Filter [codegen id : 22] -Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] -Condition : (coalesce(cs_qty#78, 0) > 0) - -(67) Sort [codegen id : 22] -Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] -Arguments: [cs_sold_year#76 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST, cs_customer_sk#77 ASC NULLS FIRST], false, 0 - -(68) SortMergeJoin [codegen id : 23] -Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77] -Join type: Inner -Join condition: None - -(69) Project [codegen id : 23] -Output [12]: [round((cast(ss_qty#24 as double) / cast(coalesce((ws_qty#51 + cs_qty#78), 1) as double)), 2) AS ratio#81, ss_qty#24 AS store_qty#82, ss_wc#25 AS store_wholesale_cost#83, ss_sp#26 AS store_sales_price#84, (coalesce(ws_qty#51, 0) + coalesce(cs_qty#78, 0)) AS other_chan_qty#85, (coalesce(ws_wc#52, 0.00) + coalesce(cs_wc#79, 0.00)) AS other_chan_wholesale_cost#86, (coalesce(ws_sp#53, 0.00) + coalesce(cs_sp#80, 0.00)) AS other_chan_sales_price#87, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, cs_qty#78] -Input [15]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53, cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] - -(70) TakeOrderedAndProject -Input [12]: [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, cs_qty#78] -Arguments: 100, [ratio#81 ASC NULLS FIRST, ss_qty#24 DESC NULLS LAST, ss_wc#25 DESC NULLS LAST, ss_sp#26 DESC NULLS LAST, other_chan_qty#85 ASC NULLS FIRST, other_chan_wholesale_cost#86 ASC NULLS FIRST, other_chan_sales_price#87 ASC NULLS FIRST, round((cast(ss_qty#24 as double) / cast(coalesce((ws_qty#51 + cs_qty#78), 1) as double)), 2) ASC NULLS FIRST], [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87] +(49) CometFilter +Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] +Condition : (isnotnull(cr_order_number#51) AND isnotnull(cr_item_sk#50)) + +(50) CometProject +Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] +Arguments: [cr_item_sk#50, cr_order_number#51], [cr_item_sk#50, cr_order_number#51] + +(51) CometColumnarExchange +Input [2]: [cr_item_sk#50, cr_order_number#51] +Arguments: hashpartitioning(cr_order_number#51, cr_item_sk#50, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(52) CometSort +Input [2]: [cr_item_sk#50, cr_order_number#51] +Arguments: [cr_item_sk#50, cr_order_number#51], [cr_order_number#51 ASC NULLS FIRST, cr_item_sk#50 ASC NULLS FIRST] + +(53) CometSortMergeJoin +Left output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] +Right output [2]: [cr_item_sk#50, cr_order_number#51] +Arguments: [cs_order_number#44, cs_item_sk#43], [cr_order_number#51, cr_item_sk#50], LeftOuter + +(54) CometFilter +Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] +Condition : isnull(cr_order_number#51) + +(55) CometProject +Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] +Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] + +(56) ReusedExchange [Reuses operator id: 15] +Output [2]: [d_date_sk#53, d_year#54] + +(57) CometBroadcastHashJoin +Left output [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] +Right output [2]: [d_date_sk#53, d_year#54] +Arguments: [cs_sold_date_sk#48], [d_date_sk#53], Inner, BuildRight + +(58) CometProject +Input [8]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, d_date_sk#53, d_year#54] +Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] + +(59) CometHashAggregate +Input [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] +Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] +Functions [3]: [partial_sum(cs_quantity#45), partial_sum(UnscaledValue(cs_wholesale_cost#46)), partial_sum(UnscaledValue(cs_sales_price#47))] + +(60) CometColumnarExchange +Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] +Arguments: hashpartitioning(d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(61) CometHashAggregate +Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] +Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] +Functions [3]: [sum(cs_quantity#45), sum(UnscaledValue(cs_wholesale_cost#46)), sum(UnscaledValue(cs_sales_price#47))] + +(62) CometFilter +Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] +Condition : (coalesce(cs_qty#60, 0) > 0) + +(63) CometSort +Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] +Arguments: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62], [cs_sold_year#58 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST, cs_customer_sk#59 ASC NULLS FIRST] + +(64) CometSortMergeJoin +Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] +Right output [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] +Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59], Inner + +(65) ColumnarToRow [codegen id : 1] +Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] + +(66) Project [codegen id : 1] +Output [12]: [round((cast(ss_qty#18 as double) / cast(coalesce((ws_qty#39 + cs_qty#60), 1) as double)), 2) AS ratio#63, ss_qty#18 AS store_qty#64, ss_wc#19 AS store_wholesale_cost#65, ss_sp#20 AS store_sales_price#66, (coalesce(ws_qty#39, 0) + coalesce(cs_qty#60, 0)) AS other_chan_qty#67, (coalesce(ws_wc#40, 0.00) + coalesce(cs_wc#61, 0.00)) AS other_chan_wholesale_cost#68, (coalesce(ws_sp#41, 0.00) + coalesce(cs_sp#62, 0.00)) AS other_chan_sales_price#69, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, cs_qty#60] +Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] + +(67) TakeOrderedAndProject +Input [12]: [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, cs_qty#60] +Arguments: 100, [ratio#63 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 DESC NULLS LAST, ss_sp#20 DESC NULLS LAST, other_chan_qty#67 ASC NULLS FIRST, other_chan_wholesale_cost#68 ASC NULLS FIRST, other_chan_sales_price#69 ASC NULLS FIRST, round((cast(ss_qty#18 as double) / cast(coalesce((ws_qty#39 + cs_qty#60), 1) as double)), 2) ASC NULLS FIRST], [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (74) -+- * ColumnarToRow (73) - +- CometFilter (72) - +- CometScan parquet spark_catalog.default.date_dim (71) +BroadcastExchange (71) ++- * ColumnarToRow (70) + +- CometFilter (69) + +- CometScan parquet spark_catalog.default.date_dim (68) -(71) Scan parquet spark_catalog.default.date_dim +(68) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#12, d_year#13] 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 -(72) CometFilter +(69) CometFilter Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(73) ColumnarToRow [codegen id : 1] +(70) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] -(74) BroadcastExchange +(71) BroadcastExchange Input [2]: [d_date_sk#12, d_year#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#48 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/simplified.txt index 13a8aecff..b7f5911e7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/simplified.txt @@ -1,127 +1,78 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ws_qty,cs_qty,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (23) + WholeStageCodegen (1) Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp] - SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,cs_sold_year,cs_item_sk,cs_customer_sk] + ColumnarToRow InputAdapter - WholeStageCodegen (15) - Project [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] - SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ws_sold_year,ws_item_sk,ws_customer_sk] - InputAdapter - WholeStageCodegen (7) - Sort [ss_sold_year,ss_item_sk,ss_customer_sk] - HashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] [sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price)),ss_sold_year,ss_qty,ss_wc,ss_sp,sum,sum,sum] - InputAdapter - Exchange [d_year,ss_item_sk,ss_customer_sk] #1 - WholeStageCodegen (6) - HashAggregate [d_year,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] [sum,sum,sum,sum,sum,sum] - Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - Filter [sr_ticket_number] - SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - WholeStageCodegen (2) - Sort [ss_ticket_number,ss_item_sk] - InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - WholeStageCodegen (4) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - Exchange [sr_ticket_number,sr_item_sk] #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - WholeStageCodegen (14) - Sort [ws_sold_year,ws_item_sk,ws_customer_sk] - Filter [ws_qty] - HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] [sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price)),ws_sold_year,ws_customer_sk,ws_qty,ws_wc,ws_sp,sum,sum,sum] - InputAdapter - Exchange [d_year,ws_item_sk,ws_bill_customer_sk] #5 - WholeStageCodegen (13) - HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] [sum,sum,sum,sum,sum,sum] - Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - Filter [wr_order_number] - SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - InputAdapter - WholeStageCodegen (9) - Sort [ws_order_number,ws_item_sk] - InputAdapter - Exchange [ws_order_number,ws_item_sk] #6 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - WholeStageCodegen (11) - Sort [wr_order_number,wr_item_sk] - InputAdapter - Exchange [wr_order_number,wr_item_sk] #7 - WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - WholeStageCodegen (22) - Sort [cs_sold_year,cs_item_sk,cs_customer_sk] - Filter [cs_qty] - HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] [sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price)),cs_sold_year,cs_customer_sk,cs_qty,cs_wc,cs_sp,sum,sum,sum] - InputAdapter - Exchange [d_year,cs_item_sk,cs_bill_customer_sk] #8 - WholeStageCodegen (21) - HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] [sum,sum,sum,sum,sum,sum] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - Filter [cr_order_number] - SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - InputAdapter - WholeStageCodegen (17) - Sort [cs_order_number,cs_item_sk] - InputAdapter - Exchange [cs_order_number,cs_item_sk] #9 - WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - WholeStageCodegen (19) - Sort [cr_order_number,cr_item_sk] - InputAdapter - Exchange [cr_order_number,cr_item_sk] #10 - WholeStageCodegen (18) + CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] + CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] + CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] + CometSort [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp] + CometHashAggregate [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,d_year,sum,sum,sum,sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price))] + CometColumnarExchange [d_year,ss_item_sk,ss_customer_sk] #1 + CometHashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum,ss_quantity,ss_wholesale_cost,ss_sales_price] + CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #2 + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometSort [sr_item_sk,sr_ticket_number] + CometColumnarExchange [sr_ticket_number,sr_item_sk] #4 + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #5 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometSort [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] + CometFilter [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] + CometHashAggregate [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp,d_year,ws_bill_customer_sk,sum,sum,sum,sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price))] + CometColumnarExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 + CometHashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum,ws_quantity,ws_wholesale_cost,ws_sales_price] + CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] + CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,d_date_sk,d_year] + CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] + CometSortMergeJoin [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] + CometSort [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometColumnarExchange [ws_order_number,ws_item_sk] #7 + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_item_sk,wr_order_number] + CometColumnarExchange [wr_order_number,wr_item_sk] #8 + CometProject [wr_item_sk,wr_order_number] + CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] + ReusedExchange [d_date_sk,d_year] #5 + CometSort [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] + CometFilter [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] + CometHashAggregate [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp,d_year,cs_bill_customer_sk,sum,sum,sum,sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price))] + CometColumnarExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 + CometHashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum,cs_quantity,cs_wholesale_cost,cs_sales_price] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] + CometSortMergeJoin [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] + CometSort [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + CometColumnarExchange [cs_order_number,cs_item_sk] #10 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [cr_item_sk,cr_order_number] + CometColumnarExchange [cr_order_number,cr_item_sk] #11 + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] + ReusedExchange [d_date_sk,d_year] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/explain.txt index 3bad7fc44..4c679d673 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/explain.txt @@ -1,35 +1,34 @@ == Physical Plan == -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) +* ColumnarToRow (30) ++- CometTakeOrderedAndProject (29) + +- CometProject (28) + +- CometBroadcastHashJoin (27) + :- CometHashAggregate (23) + : +- CometColumnarExchange (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) + +- CometBroadcastExchange (26) + +- CometFilter (25) + +- CometScan parquet spark_catalog.default.customer (24) (1) Scan parquet spark_catalog.default.store_sales @@ -133,82 +132,76 @@ Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, 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))] -(22) ColumnarToRow [codegen id : 1] +(22) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(24) HashAggregate [codegen id : 3] +(23) CometHashAggregate 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#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] -(25) Scan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] +(24) Scan parquet spark_catalog.default.customer +Output [3]: [c_customer_sk#21, c_first_name#22, c_last_name#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(26) CometFilter -Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Condition : isnotnull(c_customer_sk#25) +(25) CometFilter +Input [3]: [c_customer_sk#21, c_first_name#22, c_last_name#23] +Condition : isnotnull(c_customer_sk#21) -(27) ColumnarToRow [codegen id : 2] -Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] +(26) CometBroadcastExchange +Input [3]: [c_customer_sk#21, c_first_name#22, c_last_name#23] +Arguments: [c_customer_sk#21, c_first_name#22, c_last_name#23] -(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=2] +(27) CometBroadcastHashJoin +Left output [5]: [ss_ticket_number#5, ss_customer_sk#1, s_city#15, amt#24, profit#25] +Right output [3]: [c_customer_sk#21, c_first_name#22, c_last_name#23] +Arguments: [ss_customer_sk#1], [c_customer_sk#21], Inner, BuildRight -(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 +(28) CometProject +Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#15, amt#24, profit#25, c_customer_sk#21, c_first_name#22, c_last_name#23] +Arguments: [c_last_name#23, c_first_name#22, substr(s_city, 1, 30)#26, ss_ticket_number#5, amt#24, profit#25, s_city#15], [c_last_name#23, c_first_name#22, substr(s_city#15, 1, 30) AS substr(s_city, 1, 30)#26, ss_ticket_number#5, amt#24, profit#25, s_city#15] -(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] +(29) CometTakeOrderedAndProject +Input [7]: [c_last_name#23, c_first_name#22, substr(s_city, 1, 30)#26, ss_ticket_number#5, amt#24, profit#25, s_city#15] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#23 ASC NULLS FIRST,c_first_name#22 ASC NULLS FIRST,substr(s_city#15, 1, 30) ASC NULLS FIRST,profit#25 ASC NULLS FIRST], output=[c_last_name#23,c_first_name#22,substr(s_city, 1, 30)#26,ss_ticket_number#5,amt#24,profit#25]), [c_last_name#23, c_first_name#22, substr(s_city, 1, 30)#26, ss_ticket_number#5, amt#24, profit#25], 100, [c_last_name#23 ASC NULLS FIRST, c_first_name#22 ASC NULLS FIRST, substr(s_city#15, 1, 30) ASC NULLS FIRST, profit#25 ASC NULLS FIRST], [c_last_name#23, c_first_name#22, substr(s_city, 1, 30)#26, ss_ticket_number#5, amt#24, profit#25] -(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] +(30) ColumnarToRow [codegen id : 1] +Input [6]: [c_last_name#23, c_first_name#22, substr(s_city, 1, 30)#26, ss_ticket_number#5, amt#24, profit#25] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (36) -+- * ColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan parquet spark_catalog.default.date_dim (32) +BroadcastExchange (35) ++- * ColumnarToRow (34) + +- CometProject (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.date_dim (31) -(32) Scan parquet spark_catalog.default.date_dim +(31) 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 -(33) CometFilter +(32) 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)) -(34) CometProject +(33) CometProject Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(35) ColumnarToRow [codegen id : 1] +(34) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(36) BroadcastExchange +(35) BroadcastExchange Input [1]: [d_date_sk#10] -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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/simplified.txt index 57598ec35..6e6401470 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/simplified.txt @@ -1,46 +1,40 @@ -TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, 30),ss_ticket_number,amt] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum,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_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city,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_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_store_sk,s_city] - CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [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,d_date_sk] - CometFilter [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] - 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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [s_store_sk,s_city] #4 - CometProject [s_store_sk,s_city] - CometFilter [s_store_sk,s_number_employees,s_city] - CometScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] - CometBroadcastExchange [hd_demo_sk] #5 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [c_last_name,c_first_name,substr(s_city, 1, 30),ss_ticket_number,amt,profit,s_city] + CometProject [c_last_name,c_first_name,substr(s_city, 1, 30),ss_ticket_number,amt,profit,s_city] + CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,s_city,amt,profit,c_customer_sk,c_first_name,c_last_name] + CometHashAggregate [ss_ticket_number,ss_customer_sk,s_city,amt,profit,ss_addr_sk,sum,sum,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] + CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city] #1 + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum,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_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city,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_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_store_sk,s_city] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + CometBroadcastHashJoin [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,d_date_sk] + CometFilter [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] + 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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometBroadcastExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometBroadcastExchange [s_store_sk,s_city] #4 + CometProject [s_store_sk,s_city] + CometFilter [s_store_sk,s_number_employees,s_city] + CometScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] + CometBroadcastExchange [hd_demo_sk] #5 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #6 + CometFilter [c_customer_sk,c_first_name,c_last_name] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/explain.txt index 07bfd877c..19d7ccc99 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/explain.txt @@ -1,48 +1,46 @@ == Physical Plan == -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) +* ColumnarToRow (42) ++- CometTakeOrderedAndProject (41) + +- CometHashAggregate (40) + +- CometColumnarExchange (39) + +- CometHashAggregate (38) + +- CometProject (37) + +- CometBroadcastHashJoin (36) + :- 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 (35) + +- CometHashAggregate (34) + +- CometColumnarExchange (33) + +- CometHashAggregate (32) + +- CometBroadcastHashJoin (31) + :- CometProject (16) + : +- CometFilter (15) + : +- CometScan parquet spark_catalog.default.customer_address (14) + +- CometBroadcastExchange (30) + +- CometProject (29) + +- CometFilter (28) + +- CometHashAggregate (27) + +- CometColumnarExchange (26) + +- CometHashAggregate (25) + +- CometProject (24) + +- CometBroadcastHashJoin (23) + :- CometFilter (18) + : +- CometScan parquet spark_catalog.default.customer_address (17) + +- CometBroadcastExchange (22) + +- CometProject (21) + +- CometFilter (20) + +- CometScan parquet spark_catalog.default.customer (19) (1) Scan parquet spark_catalog.default.store_sales @@ -109,186 +107,168 @@ Arguments: [ss_store_sk#1], [s_store_sk#8], Inner, BuildRight 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] -(14) ColumnarToRow [codegen id : 5] -Input [3]: [ss_net_profit#2, s_store_name#9, s_zip#10] - -(15) Scan parquet spark_catalog.default.customer_address +(14) 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 -(16) CometFilter +(15) 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))) -(17) CometProject +(16) CometProject Input [1]: [ca_zip#11] Arguments: [ca_zip#12], [substr(ca_zip#11, 1, 5) AS ca_zip#12] -(18) ColumnarToRow [codegen id : 3] -Input [1]: [ca_zip#12] - -(19) Scan parquet spark_catalog.default.customer_address +(17) 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 -(20) CometFilter +(18) CometFilter Input [2]: [ca_address_sk#13, ca_zip#14] Condition : isnotnull(ca_address_sk#13) -(21) Scan parquet spark_catalog.default.customer +(19) 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 -(22) CometFilter +(20) 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)) -(23) CometProject +(21) CometProject Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] Arguments: [c_current_addr_sk#15], [c_current_addr_sk#15] -(24) CometBroadcastExchange +(22) CometBroadcastExchange Input [1]: [c_current_addr_sk#15] Arguments: [c_current_addr_sk#15] -(25) CometBroadcastHashJoin +(23) 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 -(26) CometProject +(24) CometProject Input [3]: [ca_address_sk#13, ca_zip#14, c_current_addr_sk#15] Arguments: [ca_zip#14], [ca_zip#14] -(27) CometHashAggregate +(25) CometHashAggregate Input [1]: [ca_zip#14] Keys [1]: [ca_zip#14] Functions [1]: [partial_count(1)] -(28) ColumnarToRow [codegen id : 1] -Input [2]: [ca_zip#14, count#17] - -(29) Exchange +(26) CometColumnarExchange Input [2]: [ca_zip#14, count#17] -Arguments: hashpartitioning(ca_zip#14, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: hashpartitioning(ca_zip#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(30) HashAggregate [codegen id : 2] +(27) CometHashAggregate Input [2]: [ca_zip#14, count#17] Keys [1]: [ca_zip#14] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#18] -Results [2]: [substr(ca_zip#14, 1, 5) AS ca_zip#19, count(1)#18 AS cnt#20] -(31) Filter [codegen id : 2] -Input [2]: [ca_zip#19, cnt#20] -Condition : (cnt#20 > 10) +(28) CometFilter +Input [2]: [ca_zip#18, cnt#19] +Condition : (cnt#19 > 10) -(32) Project [codegen id : 2] -Output [1]: [ca_zip#19] -Input [2]: [ca_zip#19, cnt#20] +(29) CometProject +Input [2]: [ca_zip#18, cnt#19] +Arguments: [ca_zip#18], [ca_zip#18] -(33) BroadcastExchange -Input [1]: [ca_zip#19] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [plan_id=2] +(30) CometBroadcastExchange +Input [1]: [ca_zip#18] +Arguments: [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 +(31) CometBroadcastHashJoin +Left output [1]: [ca_zip#12] +Right output [1]: [ca_zip#18] +Arguments: [coalesce(ca_zip#12, ), isnull(ca_zip#12)], [coalesce(ca_zip#18, ), isnull(ca_zip#18)], LeftSemi, BuildRight -(35) HashAggregate [codegen id : 3] +(32) CometHashAggregate Input [1]: [ca_zip#12] Keys [1]: [ca_zip#12] Functions: [] -Aggregate Attributes: [] -Results [1]: [ca_zip#12] -(36) Exchange +(33) CometColumnarExchange Input [1]: [ca_zip#12] -Arguments: hashpartitioning(ca_zip#12, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(ca_zip#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(37) HashAggregate [codegen id : 4] +(34) CometHashAggregate Input [1]: [ca_zip#12] Keys [1]: [ca_zip#12] Functions: [] -Aggregate Attributes: [] -Results [1]: [ca_zip#12] -(38) BroadcastExchange +(35) CometBroadcastExchange Input [1]: [ca_zip#12] -Arguments: HashedRelationBroadcastMode(List(substr(input[0, string, true], 1, 2)),false), [plan_id=4] +Arguments: [ca_zip#12] -(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 +(36) CometBroadcastHashJoin +Left output [3]: [ss_net_profit#2, s_store_name#9, s_zip#10] +Right output [1]: [ca_zip#12] +Arguments: [substr(s_zip#10, 1, 2)], [substr(ca_zip#12, 1, 2)], Inner, BuildRight -(40) Project [codegen id : 5] -Output [2]: [ss_net_profit#2, s_store_name#9] +(37) CometProject Input [4]: [ss_net_profit#2, s_store_name#9, s_zip#10, ca_zip#12] +Arguments: [ss_net_profit#2, s_store_name#9], [ss_net_profit#2, s_store_name#9] -(41) HashAggregate [codegen id : 5] +(38) CometHashAggregate 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#21] -Results [2]: [s_store_name#9, sum#22] -(42) Exchange -Input [2]: [s_store_name#9, sum#22] -Arguments: hashpartitioning(s_store_name#9, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(39) CometColumnarExchange +Input [2]: [s_store_name#9, sum#20] +Arguments: hashpartitioning(s_store_name#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(43) HashAggregate [codegen id : 6] -Input [2]: [s_store_name#9, sum#22] +(40) CometHashAggregate +Input [2]: [s_store_name#9, sum#20] Keys [1]: [s_store_name#9] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -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] -(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] +(41) CometTakeOrderedAndProject +Input [2]: [s_store_name#9, sum(ss_net_profit)#21] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#9 ASC NULLS FIRST], output=[s_store_name#9,sum(ss_net_profit)#21]), [s_store_name#9, sum(ss_net_profit)#21], 100, [s_store_name#9 ASC NULLS FIRST], [s_store_name#9, sum(ss_net_profit)#21] + +(42) ColumnarToRow [codegen id : 1] +Input [2]: [s_store_name#9, sum(ss_net_profit)#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (49) -+- * ColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan parquet spark_catalog.default.date_dim (45) +BroadcastExchange (47) ++- * ColumnarToRow (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) -(45) Scan parquet spark_catalog.default.date_dim +(43) 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 -(46) CometFilter +(44) 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)) -(47) CometProject +(45) CometProject Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Arguments: [d_date_sk#5], [d_date_sk#5] -(48) ColumnarToRow [codegen id : 1] +(46) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(49) BroadcastExchange +(47) BroadcastExchange Input [1]: [d_date_sk#5] -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-spark3_5/q8/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/simplified.txt index 637f3b7ba..6e1a3d332 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/simplified.txt @@ -1,66 +1,52 @@ -TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] - WholeStageCodegen (6) - HashAggregate [s_store_name,sum] [sum(UnscaledValue(ss_net_profit)),sum(ss_net_profit),sum] - InputAdapter - Exchange [s_store_name] #1 - WholeStageCodegen (5) - HashAggregate [s_store_name,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_store_name] - BroadcastHashJoin [s_zip,ca_zip] - ColumnarToRow - InputAdapter - CometProject [ss_net_profit,s_store_name,s_zip] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,s_store_sk,s_store_name,s_zip] - CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #4 - CometFilter [s_store_sk,s_store_name,s_zip] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - HashAggregate [ca_zip] - InputAdapter - Exchange [ca_zip] #6 - WholeStageCodegen (3) - HashAggregate [ca_zip] - BroadcastHashJoin [ca_zip,ca_zip] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [s_store_name,sum(ss_net_profit)] + CometHashAggregate [s_store_name,sum(ss_net_profit),sum,sum(UnscaledValue(ss_net_profit))] + CometColumnarExchange [s_store_name] #1 + CometHashAggregate [s_store_name,sum,ss_net_profit] + CometProject [ss_net_profit,s_store_name] + CometBroadcastHashJoin [ss_net_profit,s_store_name,s_zip,ca_zip] + CometProject [ss_net_profit,s_store_name,s_zip] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,s_store_sk,s_store_name,s_zip] + CometProject [ss_store_sk,ss_net_profit] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [ca_zip] [ca_zip] - CometFilter [ca_zip] - CometScan parquet spark_catalog.default.customer_address [ca_zip] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - Project [ca_zip] - Filter [cnt] - HashAggregate [ca_zip,count] [count(1),ca_zip,cnt,count] - InputAdapter - Exchange [ca_zip] #8 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ca_zip,count] - CometProject [ca_zip] - CometBroadcastHashJoin [ca_address_sk,ca_zip,c_current_addr_sk] - CometFilter [ca_address_sk,ca_zip] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange [c_current_addr_sk] #9 - CometProject [c_current_addr_sk] - CometFilter [c_current_addr_sk,c_preferred_cust_flag] - CometScan parquet spark_catalog.default.customer [c_current_addr_sk,c_preferred_cust_flag] + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #4 + CometFilter [s_store_sk,s_store_name,s_zip] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] + CometBroadcastExchange [ca_zip] #5 + CometHashAggregate [ca_zip] + CometColumnarExchange [ca_zip] #6 + CometHashAggregate [ca_zip] + CometBroadcastHashJoin [ca_zip,ca_zip] + CometProject [ca_zip] [ca_zip] + CometFilter [ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_zip] + CometBroadcastExchange [ca_zip] #7 + CometProject [ca_zip] + CometFilter [ca_zip,cnt] + CometHashAggregate [ca_zip,cnt,ca_zip,count,count(1)] + CometColumnarExchange [ca_zip] #8 + CometHashAggregate [ca_zip,count] + CometProject [ca_zip] + CometBroadcastHashJoin [ca_address_sk,ca_zip,c_current_addr_sk] + CometFilter [ca_address_sk,ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] + CometBroadcastExchange [c_current_addr_sk] #9 + CometProject [c_current_addr_sk] + CometFilter [c_current_addr_sk,c_preferred_cust_flag] + 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-spark3_5/q80/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/explain.txt index db2d015db..a66d4f66d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/explain.txt @@ -1,111 +1,108 @@ == Physical Plan == -TakeOrderedAndProject (107) -+- * HashAggregate (106) - +- Exchange (105) - +- * HashAggregate (104) - +- * Expand (103) - +- Union (102) - :- * HashAggregate (39) - : +- Exchange (38) - : +- * HashAggregate (37) - : +- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * Project (29) - : : +- * BroadcastHashJoin Inner BuildRight (28) - : : :- * Project (22) - : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : :- * Project (16) - : : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : : :- * Project (13) - : : : : : +- * SortMergeJoin LeftOuter (12) - : : : : : :- * Sort (5) - : : : : : : +- Exchange (4) - : : : : : : +- * ColumnarToRow (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : +- * Sort (11) - : : : : : +- Exchange (10) - : : : : : +- * ColumnarToRow (9) - : : : : : +- CometProject (8) - : : : : : +- CometFilter (7) - : : : : : +- CometScan parquet spark_catalog.default.store_returns (6) - : : : : +- ReusedExchange (14) - : : : +- BroadcastExchange (20) - : : : +- * ColumnarToRow (19) - : : : +- CometFilter (18) - : : : +- CometScan parquet spark_catalog.default.store (17) - : : +- BroadcastExchange (27) - : : +- * ColumnarToRow (26) - : : +- CometProject (25) - : : +- CometFilter (24) - : : +- CometScan parquet spark_catalog.default.item (23) - : +- BroadcastExchange (34) - : +- * ColumnarToRow (33) - : +- CometProject (32) - : +- CometFilter (31) - : +- CometScan parquet spark_catalog.default.promotion (30) - :- * HashAggregate (70) - : +- Exchange (69) - : +- * HashAggregate (68) - : +- * Project (67) - : +- * BroadcastHashJoin Inner BuildRight (66) - : :- * Project (64) - : : +- * BroadcastHashJoin Inner BuildRight (63) - : : :- * Project (61) - : : : +- * BroadcastHashJoin Inner BuildRight (60) - : : : :- * Project (55) - : : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : : :- * Project (52) - : : : : : +- * SortMergeJoin LeftOuter (51) - : : : : : :- * Sort (44) - : : : : : : +- Exchange (43) - : : : : : : +- * ColumnarToRow (42) - : : : : : : +- CometFilter (41) - : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (40) - : : : : : +- * Sort (50) - : : : : : +- Exchange (49) - : : : : : +- * ColumnarToRow (48) - : : : : : +- CometProject (47) - : : : : : +- CometFilter (46) - : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (45) - : : : : +- ReusedExchange (53) - : : : +- BroadcastExchange (59) - : : : +- * ColumnarToRow (58) - : : : +- CometFilter (57) - : : : +- CometScan parquet spark_catalog.default.catalog_page (56) - : : +- ReusedExchange (62) - : +- ReusedExchange (65) - +- * HashAggregate (101) - +- Exchange (100) - +- * HashAggregate (99) - +- * Project (98) - +- * BroadcastHashJoin Inner BuildRight (97) - :- * Project (95) - : +- * BroadcastHashJoin Inner BuildRight (94) - : :- * Project (92) - : : +- * BroadcastHashJoin Inner BuildRight (91) - : : :- * Project (86) - : : : +- * BroadcastHashJoin Inner BuildRight (85) - : : : :- * Project (83) - : : : : +- * SortMergeJoin LeftOuter (82) - : : : : :- * Sort (75) - : : : : : +- Exchange (74) - : : : : : +- * ColumnarToRow (73) - : : : : : +- CometFilter (72) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (71) - : : : : +- * Sort (81) - : : : : +- Exchange (80) - : : : : +- * ColumnarToRow (79) - : : : : +- CometProject (78) - : : : : +- CometFilter (77) - : : : : +- CometScan parquet spark_catalog.default.web_returns (76) - : : : +- ReusedExchange (84) - : : +- BroadcastExchange (90) - : : +- * ColumnarToRow (89) - : : +- CometFilter (88) - : : +- CometScan parquet spark_catalog.default.web_site (87) - : +- ReusedExchange (93) - +- ReusedExchange (96) +TakeOrderedAndProject (104) ++- * HashAggregate (103) + +- * ColumnarToRow (102) + +- CometColumnarExchange (101) + +- RowToColumnar (100) + +- * HashAggregate (99) + +- * Expand (98) + +- Union (97) + :- * HashAggregate (38) + : +- * ColumnarToRow (37) + : +- CometColumnarExchange (36) + : +- CometHashAggregate (35) + : +- CometProject (34) + : +- CometBroadcastHashJoin (33) + : :- CometProject (28) + : : +- CometBroadcastHashJoin (27) + : : :- CometProject (22) + : : : +- CometBroadcastHashJoin (21) + : : : :- CometProject (17) + : : : : +- CometBroadcastHashJoin (16) + : : : : :- CometProject (11) + : : : : : +- CometSortMergeJoin (10) + : : : : : :- CometSort (4) + : : : : : : +- CometColumnarExchange (3) + : : : : : : +- CometFilter (2) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : +- CometSort (9) + : : : : : +- CometColumnarExchange (8) + : : : : : +- CometProject (7) + : : : : : +- CometFilter (6) + : : : : : +- CometScan parquet spark_catalog.default.store_returns (5) + : : : : +- CometBroadcastExchange (15) + : : : : +- CometProject (14) + : : : : +- CometFilter (13) + : : : : +- CometScan parquet spark_catalog.default.date_dim (12) + : : : +- CometBroadcastExchange (20) + : : : +- CometFilter (19) + : : : +- CometScan parquet spark_catalog.default.store (18) + : : +- CometBroadcastExchange (26) + : : +- CometProject (25) + : : +- CometFilter (24) + : : +- CometScan parquet spark_catalog.default.item (23) + : +- CometBroadcastExchange (32) + : +- CometProject (31) + : +- CometFilter (30) + : +- CometScan parquet spark_catalog.default.promotion (29) + :- * HashAggregate (67) + : +- * ColumnarToRow (66) + : +- CometColumnarExchange (65) + : +- CometHashAggregate (64) + : +- CometProject (63) + : +- CometBroadcastHashJoin (62) + : :- CometProject (60) + : : +- CometBroadcastHashJoin (59) + : : :- CometProject (57) + : : : +- CometBroadcastHashJoin (56) + : : : :- CometProject (52) + : : : : +- CometBroadcastHashJoin (51) + : : : : :- CometProject (49) + : : : : : +- CometSortMergeJoin (48) + : : : : : :- CometSort (42) + : : : : : : +- CometColumnarExchange (41) + : : : : : : +- CometFilter (40) + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (39) + : : : : : +- CometSort (47) + : : : : : +- CometColumnarExchange (46) + : : : : : +- CometProject (45) + : : : : : +- CometFilter (44) + : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (43) + : : : : +- ReusedExchange (50) + : : : +- CometBroadcastExchange (55) + : : : +- CometFilter (54) + : : : +- CometScan parquet spark_catalog.default.catalog_page (53) + : : +- ReusedExchange (58) + : +- ReusedExchange (61) + +- * HashAggregate (96) + +- * ColumnarToRow (95) + +- CometColumnarExchange (94) + +- CometHashAggregate (93) + +- CometProject (92) + +- CometBroadcastHashJoin (91) + :- CometProject (89) + : +- CometBroadcastHashJoin (88) + : :- CometProject (86) + : : +- CometBroadcastHashJoin (85) + : : :- CometProject (81) + : : : +- CometBroadcastHashJoin (80) + : : : :- CometProject (78) + : : : : +- CometSortMergeJoin (77) + : : : : :- CometSort (71) + : : : : : +- CometColumnarExchange (70) + : : : : : +- CometFilter (69) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (68) + : : : : +- CometSort (76) + : : : : +- CometColumnarExchange (75) + : : : : +- CometProject (74) + : : : : +- CometFilter (73) + : : : : +- CometScan parquet spark_catalog.default.web_returns (72) + : : : +- ReusedExchange (79) + : : +- CometBroadcastExchange (84) + : : +- CometFilter (83) + : : +- CometScan parquet spark_catalog.default.web_site (82) + : +- ReusedExchange (87) + +- ReusedExchange (90) (1) Scan parquet spark_catalog.default.store_sales @@ -120,526 +117,503 @@ ReadSchema: struct -(7) CometFilter +(6) CometFilter Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] Condition : (isnotnull(sr_item_sk#9) AND isnotnull(sr_ticket_number#10)) -(8) CometProject +(7) CometProject Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -(9) ColumnarToRow [codegen id : 3] -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] - -(10) Exchange +(8) CometColumnarExchange Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(11) Sort [codegen id : 4] +(9) CometSort Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST], false, 0 +Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST] -(12) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_item_sk#1, ss_ticket_number#4] -Right keys [2]: [sr_item_sk#9, sr_ticket_number#10] -Join type: LeftOuter -Join condition: None +(10) CometSortMergeJoin +Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] +Right output [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] +Arguments: [ss_item_sk#1, ss_ticket_number#4], [sr_item_sk#9, sr_ticket_number#10], LeftOuter -(13) Project [codegen id : 9] -Output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] +(11) CometProject Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] +Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] -(14) ReusedExchange [Reuses operator id: 112] -Output [1]: [d_date_sk#14] +(12) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#15] +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-22), IsNotNull(d_date_sk)] +ReadSchema: struct -(15) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None +(13) CometFilter +Input [2]: [d_date_sk#14, d_date#15] +Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#15 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) -(16) Project [codegen id : 9] -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] +(14) CometProject +Input [2]: [d_date_sk#14, d_date#15] +Arguments: [d_date_sk#14], [d_date_sk#14] + +(15) CometBroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: [d_date_sk#14] + +(16) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] +Right output [1]: [d_date_sk#14] +Arguments: [ss_sold_date_sk#7], [d_date_sk#14], Inner, BuildRight + +(17) CometProject Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#14] +Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] -(17) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#15, s_store_id#16] +(18) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#16, s_store_id#17] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(18) CometFilter -Input [2]: [s_store_sk#15, s_store_id#16] -Condition : isnotnull(s_store_sk#15) - -(19) ColumnarToRow [codegen id : 6] -Input [2]: [s_store_sk#15, s_store_id#16] +(19) CometFilter +Input [2]: [s_store_sk#16, s_store_id#17] +Condition : isnotnull(s_store_sk#16) -(20) BroadcastExchange -Input [2]: [s_store_sk#15, s_store_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(20) CometBroadcastExchange +Input [2]: [s_store_sk#16, s_store_id#17] +Arguments: [s_store_sk#16, s_store_id#17] -(21) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#15] -Join type: Inner -Join condition: None +(21) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] +Right output [2]: [s_store_sk#16, s_store_id#17] +Arguments: [ss_store_sk#2], [s_store_sk#16], Inner, BuildRight -(22) Project [codegen id : 9] -Output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#15, s_store_id#16] +(22) CometProject +Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#16, s_store_id#17] +Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17], [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] (23) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#17, i_current_price#18] +Output [2]: [i_item_sk#18, i_current_price#19] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), GreaterThan(i_current_price,50.00), IsNotNull(i_item_sk)] ReadSchema: struct (24) CometFilter -Input [2]: [i_item_sk#17, i_current_price#18] -Condition : ((isnotnull(i_current_price#18) AND (i_current_price#18 > 50.00)) AND isnotnull(i_item_sk#17)) +Input [2]: [i_item_sk#18, i_current_price#19] +Condition : ((isnotnull(i_current_price#19) AND (i_current_price#19 > 50.00)) AND isnotnull(i_item_sk#18)) (25) CometProject -Input [2]: [i_item_sk#17, i_current_price#18] -Arguments: [i_item_sk#17], [i_item_sk#17] - -(26) ColumnarToRow [codegen id : 7] -Input [1]: [i_item_sk#17] +Input [2]: [i_item_sk#18, i_current_price#19] +Arguments: [i_item_sk#18], [i_item_sk#18] -(27) BroadcastExchange -Input [1]: [i_item_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(26) CometBroadcastExchange +Input [1]: [i_item_sk#18] +Arguments: [i_item_sk#18] -(28) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#17] -Join type: Inner -Join condition: None +(27) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] +Right output [1]: [i_item_sk#18] +Arguments: [ss_item_sk#1], [i_item_sk#18], Inner, BuildRight -(29) Project [codegen id : 9] -Output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16, i_item_sk#17] +(28) CometProject +Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17, i_item_sk#18] +Arguments: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17], [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] -(30) Scan parquet spark_catalog.default.promotion -Output [2]: [p_promo_sk#19, p_channel_tv#20] +(29) Scan parquet spark_catalog.default.promotion +Output [2]: [p_promo_sk#20, p_channel_tv#21] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_channel_tv), EqualTo(p_channel_tv,N), IsNotNull(p_promo_sk)] ReadSchema: struct -(31) CometFilter -Input [2]: [p_promo_sk#19, p_channel_tv#20] -Condition : ((isnotnull(p_channel_tv#20) AND (p_channel_tv#20 = N)) AND isnotnull(p_promo_sk#19)) +(30) CometFilter +Input [2]: [p_promo_sk#20, p_channel_tv#21] +Condition : ((isnotnull(p_channel_tv#21) AND (p_channel_tv#21 = N)) AND isnotnull(p_promo_sk#20)) -(32) CometProject -Input [2]: [p_promo_sk#19, p_channel_tv#20] -Arguments: [p_promo_sk#19], [p_promo_sk#19] +(31) CometProject +Input [2]: [p_promo_sk#20, p_channel_tv#21] +Arguments: [p_promo_sk#20], [p_promo_sk#20] -(33) ColumnarToRow [codegen id : 8] -Input [1]: [p_promo_sk#19] +(32) CometBroadcastExchange +Input [1]: [p_promo_sk#20] +Arguments: [p_promo_sk#20] -(34) BroadcastExchange -Input [1]: [p_promo_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(33) CometBroadcastHashJoin +Left output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] +Right output [1]: [p_promo_sk#20] +Arguments: [ss_promo_sk#3], [p_promo_sk#20], Inner, BuildRight -(35) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_promo_sk#3] -Right keys [1]: [p_promo_sk#19] -Join type: Inner -Join condition: None +(34) CometProject +Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17, p_promo_sk#20] +Arguments: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17], [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] -(36) Project [codegen id : 9] -Output [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] -Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16, p_promo_sk#19] - -(37) HashAggregate [codegen id : 9] -Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] -Keys [1]: [s_store_id#16] +(35) CometHashAggregate +Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] +Keys [1]: [s_store_id#17] Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] -Aggregate Attributes [5]: [sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -Results [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] -(38) Exchange -Input [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] -Arguments: hashpartitioning(s_store_id#16, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(36) CometColumnarExchange +Input [6]: [s_store_id#17, sum#22, sum#23, isEmpty#24, sum#25, isEmpty#26] +Arguments: hashpartitioning(s_store_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(37) ColumnarToRow [codegen id : 1] +Input [6]: [s_store_id#17, sum#22, sum#23, isEmpty#24, sum#25, isEmpty#26] -(39) HashAggregate [codegen id : 10] -Input [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] -Keys [1]: [s_store_id#16] +(38) HashAggregate [codegen id : 1] +Input [6]: [s_store_id#17, sum#22, sum#23, isEmpty#24, sum#25, isEmpty#26] +Keys [1]: [s_store_id#17] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#5))#31, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#32, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#33] -Results [5]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#31,17,2) AS sales#34, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#32 AS returns#35, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#33 AS profit#36, store channel AS channel#37, concat(store, s_store_id#16) AS id#38] +Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#5))#27, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#28, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#29] +Results [5]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#27,17,2) AS sales#30, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#28 AS returns#31, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#29 AS profit#32, store channel AS channel#33, concat(store, s_store_id#17) AS id#34] -(40) Scan parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +(39) Scan parquet spark_catalog.default.catalog_sales +Output [7]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_order_number#38, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41] 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#41), dynamicpruningexpression(cs_sold_date_sk#41 IN dynamicpruning#42)] PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] ReadSchema: struct -(41) CometFilter -Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Condition : ((isnotnull(cs_catalog_page_sk#39) AND isnotnull(cs_item_sk#40)) AND isnotnull(cs_promo_sk#41)) +(40) CometFilter +Input [7]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_order_number#38, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41] +Condition : ((isnotnull(cs_catalog_page_sk#35) AND isnotnull(cs_item_sk#36)) AND isnotnull(cs_promo_sk#37)) -(42) ColumnarToRow [codegen id : 11] -Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +(41) CometColumnarExchange +Input [7]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_order_number#38, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41] +Arguments: hashpartitioning(cs_item_sk#36, cs_order_number#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(43) Exchange -Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Arguments: hashpartitioning(cs_item_sk#40, cs_order_number#42, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(42) CometSort +Input [7]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_order_number#38, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41] +Arguments: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_order_number#38, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41], [cs_item_sk#36 ASC NULLS FIRST, cs_order_number#38 ASC NULLS FIRST] -(44) Sort [codegen id : 12] -Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Arguments: [cs_item_sk#40 ASC NULLS FIRST, cs_order_number#42 ASC NULLS FIRST], false, 0 - -(45) Scan parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] +(43) Scan parquet spark_catalog.default.catalog_returns +Output [5]: [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46, cr_returned_date_sk#47] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(46) CometFilter -Input [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] -Condition : (isnotnull(cr_item_sk#47) AND isnotnull(cr_order_number#48)) - -(47) CometProject -Input [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] -Arguments: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50], [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] +(44) CometFilter +Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46, cr_returned_date_sk#47] +Condition : (isnotnull(cr_item_sk#43) AND isnotnull(cr_order_number#44)) -(48) ColumnarToRow [codegen id : 13] -Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] +(45) CometProject +Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46, cr_returned_date_sk#47] +Arguments: [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46], [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46] -(49) Exchange -Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -Arguments: hashpartitioning(cr_item_sk#47, cr_order_number#48, 5), ENSURE_REQUIREMENTS, [plan_id=8] +(46) CometColumnarExchange +Input [4]: [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46] +Arguments: hashpartitioning(cr_item_sk#43, cr_order_number#44, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(50) Sort [codegen id : 14] -Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -Arguments: [cr_item_sk#47 ASC NULLS FIRST, cr_order_number#48 ASC NULLS FIRST], false, 0 +(47) CometSort +Input [4]: [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46] +Arguments: [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46], [cr_item_sk#43 ASC NULLS FIRST, cr_order_number#44 ASC NULLS FIRST] -(51) SortMergeJoin [codegen id : 19] -Left keys [2]: [cs_item_sk#40, cs_order_number#42] -Right keys [2]: [cr_item_sk#47, cr_order_number#48] -Join type: LeftOuter -Join condition: None +(48) CometSortMergeJoin +Left output [7]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_order_number#38, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41] +Right output [4]: [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46] +Arguments: [cs_item_sk#36, cs_order_number#38], [cr_item_sk#43, cr_order_number#44], LeftOuter -(52) Project [codegen id : 19] -Output [8]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50] -Input [11]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] +(49) CometProject +Input [11]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_order_number#38, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41, cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46] +Arguments: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41, cr_return_amount#45, cr_net_loss#46], [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41, cr_return_amount#45, cr_net_loss#46] -(53) ReusedExchange [Reuses operator id: 112] -Output [1]: [d_date_sk#52] +(50) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#48] -(54) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_sold_date_sk#45] -Right keys [1]: [d_date_sk#52] -Join type: Inner -Join condition: None +(51) CometBroadcastHashJoin +Left output [8]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41, cr_return_amount#45, cr_net_loss#46] +Right output [1]: [d_date_sk#48] +Arguments: [cs_sold_date_sk#41], [d_date_sk#48], Inner, BuildRight -(55) Project [codegen id : 19] -Output [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50] -Input [9]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50, d_date_sk#52] +(52) CometProject +Input [9]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41, cr_return_amount#45, cr_net_loss#46, d_date_sk#48] +Arguments: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46], [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46] -(56) Scan parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +(53) Scan parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#49, cp_catalog_page_id#50] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(57) CometFilter -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Condition : isnotnull(cp_catalog_page_sk#53) - -(58) ColumnarToRow [codegen id : 16] -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] - -(59) BroadcastExchange -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -(60) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_catalog_page_sk#39] -Right keys [1]: [cp_catalog_page_sk#53] -Join type: Inner -Join condition: None - -(61) Project [codegen id : 19] -Output [7]: [cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] -Input [9]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_sk#53, cp_catalog_page_id#54] - -(62) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#55] - -(63) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_item_sk#40] -Right keys [1]: [i_item_sk#55] -Join type: Inner -Join condition: None - -(64) Project [codegen id : 19] -Output [6]: [cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] -Input [8]: [cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54, i_item_sk#55] - -(65) ReusedExchange [Reuses operator id: 34] -Output [1]: [p_promo_sk#56] - -(66) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_promo_sk#41] -Right keys [1]: [p_promo_sk#56] -Join type: Inner -Join condition: None - -(67) Project [codegen id : 19] -Output [5]: [cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] -Input [7]: [cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54, p_promo_sk#56] - -(68) HashAggregate [codegen id : 19] -Input [5]: [cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] -Keys [1]: [cp_catalog_page_id#54] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#43)), partial_sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))] -Aggregate Attributes [5]: [sum#57, sum#58, isEmpty#59, sum#60, isEmpty#61] -Results [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] - -(69) Exchange -Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] -Arguments: hashpartitioning(cp_catalog_page_id#54, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(70) HashAggregate [codegen id : 20] -Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] -Keys [1]: [cp_catalog_page_id#54] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#43)), sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00)), sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#43))#67, sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00))#68, sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))#69] -Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#43))#67,17,2) AS sales#70, sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00))#68 AS returns#71, sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))#69 AS profit#72, catalog channel AS channel#73, concat(catalog_page, cp_catalog_page_id#54) AS id#74] - -(71) Scan parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] +(54) CometFilter +Input [2]: [cp_catalog_page_sk#49, cp_catalog_page_id#50] +Condition : isnotnull(cp_catalog_page_sk#49) + +(55) CometBroadcastExchange +Input [2]: [cp_catalog_page_sk#49, cp_catalog_page_id#50] +Arguments: [cp_catalog_page_sk#49, cp_catalog_page_id#50] + +(56) CometBroadcastHashJoin +Left output [7]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46] +Right output [2]: [cp_catalog_page_sk#49, cp_catalog_page_id#50] +Arguments: [cs_catalog_page_sk#35], [cp_catalog_page_sk#49], Inner, BuildRight + +(57) CometProject +Input [9]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_sk#49, cp_catalog_page_id#50] +Arguments: [cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50], [cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50] + +(58) ReusedExchange [Reuses operator id: 26] +Output [1]: [i_item_sk#51] + +(59) CometBroadcastHashJoin +Left output [7]: [cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50] +Right output [1]: [i_item_sk#51] +Arguments: [cs_item_sk#36], [i_item_sk#51], Inner, BuildRight + +(60) CometProject +Input [8]: [cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50, i_item_sk#51] +Arguments: [cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50], [cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50] + +(61) ReusedExchange [Reuses operator id: 32] +Output [1]: [p_promo_sk#52] + +(62) CometBroadcastHashJoin +Left output [6]: [cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50] +Right output [1]: [p_promo_sk#52] +Arguments: [cs_promo_sk#37], [p_promo_sk#52], Inner, BuildRight + +(63) CometProject +Input [7]: [cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50, p_promo_sk#52] +Arguments: [cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50], [cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50] + +(64) CometHashAggregate +Input [5]: [cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50] +Keys [1]: [cp_catalog_page_id#50] +Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#39)), partial_sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#40 - coalesce(cast(cr_net_loss#46 as decimal(12,2)), 0.00)))] + +(65) CometColumnarExchange +Input [6]: [cp_catalog_page_id#50, sum#53, sum#54, isEmpty#55, sum#56, isEmpty#57] +Arguments: hashpartitioning(cp_catalog_page_id#50, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(66) ColumnarToRow [codegen id : 2] +Input [6]: [cp_catalog_page_id#50, sum#53, sum#54, isEmpty#55, sum#56, isEmpty#57] + +(67) HashAggregate [codegen id : 2] +Input [6]: [cp_catalog_page_id#50, sum#53, sum#54, isEmpty#55, sum#56, isEmpty#57] +Keys [1]: [cp_catalog_page_id#50] +Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#39)), sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00)), sum((cs_net_profit#40 - coalesce(cast(cr_net_loss#46 as decimal(12,2)), 0.00)))] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#39))#58, sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00))#59, sum((cs_net_profit#40 - coalesce(cast(cr_net_loss#46 as decimal(12,2)), 0.00)))#60] +Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#39))#58,17,2) AS sales#61, sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00))#59 AS returns#62, sum((cs_net_profit#40 - coalesce(cast(cr_net_loss#46 as decimal(12,2)), 0.00)))#60 AS profit#63, catalog channel AS channel#64, concat(catalog_page, cp_catalog_page_id#50) AS id#65] + +(68) Scan parquet spark_catalog.default.web_sales +Output [7]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_order_number#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] 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#72), dynamicpruningexpression(ws_sold_date_sk#72 IN dynamicpruning#73)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct -(72) CometFilter -Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Condition : ((isnotnull(ws_web_site_sk#76) AND isnotnull(ws_item_sk#75)) AND isnotnull(ws_promo_sk#77)) +(69) CometFilter +Input [7]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_order_number#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Condition : ((isnotnull(ws_web_site_sk#67) AND isnotnull(ws_item_sk#66)) AND isnotnull(ws_promo_sk#68)) -(73) ColumnarToRow [codegen id : 21] -Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] +(70) CometColumnarExchange +Input [7]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_order_number#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Arguments: hashpartitioning(ws_item_sk#66, ws_order_number#69, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(74) Exchange -Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Arguments: hashpartitioning(ws_item_sk#75, ws_order_number#78, 5), ENSURE_REQUIREMENTS, [plan_id=11] +(71) CometSort +Input [7]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_order_number#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Arguments: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_order_number#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72], [ws_item_sk#66 ASC NULLS FIRST, ws_order_number#69 ASC NULLS FIRST] -(75) Sort [codegen id : 22] -Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Arguments: [ws_item_sk#75 ASC NULLS FIRST, ws_order_number#78 ASC NULLS FIRST], false, 0 - -(76) Scan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +(72) Scan parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77, wr_returned_date_sk#78] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct -(77) CometFilter -Input [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] -Condition : (isnotnull(wr_item_sk#83) AND isnotnull(wr_order_number#84)) - -(78) CometProject -Input [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] -Arguments: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86], [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] +(73) CometFilter +Input [5]: [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77, wr_returned_date_sk#78] +Condition : (isnotnull(wr_item_sk#74) AND isnotnull(wr_order_number#75)) -(79) ColumnarToRow [codegen id : 23] -Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] +(74) CometProject +Input [5]: [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77, wr_returned_date_sk#78] +Arguments: [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77], [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77] -(80) Exchange -Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -Arguments: hashpartitioning(wr_item_sk#83, wr_order_number#84, 5), ENSURE_REQUIREMENTS, [plan_id=12] +(75) CometColumnarExchange +Input [4]: [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77] +Arguments: hashpartitioning(wr_item_sk#74, wr_order_number#75, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(81) Sort [codegen id : 24] -Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -Arguments: [wr_item_sk#83 ASC NULLS FIRST, wr_order_number#84 ASC NULLS FIRST], false, 0 +(76) CometSort +Input [4]: [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77] +Arguments: [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77], [wr_item_sk#74 ASC NULLS FIRST, wr_order_number#75 ASC NULLS FIRST] -(82) SortMergeJoin [codegen id : 29] -Left keys [2]: [ws_item_sk#75, ws_order_number#78] -Right keys [2]: [wr_item_sk#83, wr_order_number#84] -Join type: LeftOuter -Join condition: None +(77) CometSortMergeJoin +Left output [7]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_order_number#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Right output [4]: [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77] +Arguments: [ws_item_sk#66, ws_order_number#69], [wr_item_sk#74, wr_order_number#75], LeftOuter -(83) Project [codegen id : 29] -Output [8]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86] -Input [11]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] +(78) CometProject +Input [11]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_order_number#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77] +Arguments: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, wr_return_amt#76, wr_net_loss#77], [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, wr_return_amt#76, wr_net_loss#77] -(84) ReusedExchange [Reuses operator id: 112] -Output [1]: [d_date_sk#88] +(79) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#79] -(85) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_sold_date_sk#81] -Right keys [1]: [d_date_sk#88] -Join type: Inner -Join condition: None +(80) CometBroadcastHashJoin +Left output [8]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, wr_return_amt#76, wr_net_loss#77] +Right output [1]: [d_date_sk#79] +Arguments: [ws_sold_date_sk#72], [d_date_sk#79], Inner, BuildRight -(86) Project [codegen id : 29] -Output [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86] -Input [9]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86, d_date_sk#88] +(81) CometProject +Input [9]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, wr_return_amt#76, wr_net_loss#77, d_date_sk#79] +Arguments: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77], [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77] -(87) Scan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#89, web_site_id#90] +(82) Scan parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#80, web_site_id#81] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(88) CometFilter -Input [2]: [web_site_sk#89, web_site_id#90] -Condition : isnotnull(web_site_sk#89) - -(89) ColumnarToRow [codegen id : 26] -Input [2]: [web_site_sk#89, web_site_id#90] - -(90) BroadcastExchange -Input [2]: [web_site_sk#89, web_site_id#90] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] - -(91) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_web_site_sk#76] -Right keys [1]: [web_site_sk#89] -Join type: Inner -Join condition: None - -(92) Project [codegen id : 29] -Output [7]: [ws_item_sk#75, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] -Input [9]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_sk#89, web_site_id#90] - -(93) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#91] - -(94) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_item_sk#75] -Right keys [1]: [i_item_sk#91] -Join type: Inner -Join condition: None - -(95) Project [codegen id : 29] -Output [6]: [ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] -Input [8]: [ws_item_sk#75, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90, i_item_sk#91] - -(96) ReusedExchange [Reuses operator id: 34] -Output [1]: [p_promo_sk#92] - -(97) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_promo_sk#77] -Right keys [1]: [p_promo_sk#92] -Join type: Inner -Join condition: None - -(98) Project [codegen id : 29] -Output [5]: [ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] -Input [7]: [ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90, p_promo_sk#92] - -(99) HashAggregate [codegen id : 29] -Input [5]: [ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] -Keys [1]: [web_site_id#90] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#79)), partial_sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))] -Aggregate Attributes [5]: [sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] -Results [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] - -(100) Exchange -Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -Arguments: hashpartitioning(web_site_id#90, 5), ENSURE_REQUIREMENTS, [plan_id=14] - -(101) HashAggregate [codegen id : 30] -Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -Keys [1]: [web_site_id#90] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#79)), sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00)), sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#79))#103, sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00))#104, sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))#105] -Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#79))#103,17,2) AS sales#106, sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00))#104 AS returns#107, sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))#105 AS profit#108, web channel AS channel#109, concat(web_site, web_site_id#90) AS id#110] - -(102) Union - -(103) Expand [codegen id : 31] -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#111, id#112, spark_grouping_id#113] - -(104) HashAggregate [codegen id : 31] -Input [6]: [sales#34, returns#35, profit#36, channel#111, id#112, spark_grouping_id#113] -Keys [3]: [channel#111, id#112, spark_grouping_id#113] -Functions [3]: [partial_sum(sales#34), partial_sum(returns#35), partial_sum(profit#36)] -Aggregate Attributes [6]: [sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] -Results [9]: [channel#111, id#112, spark_grouping_id#113, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] - -(105) Exchange -Input [9]: [channel#111, id#112, spark_grouping_id#113, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] -Arguments: hashpartitioning(channel#111, id#112, spark_grouping_id#113, 5), ENSURE_REQUIREMENTS, [plan_id=15] - -(106) HashAggregate [codegen id : 32] -Input [9]: [channel#111, id#112, spark_grouping_id#113, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] -Keys [3]: [channel#111, id#112, spark_grouping_id#113] -Functions [3]: [sum(sales#34), sum(returns#35), sum(profit#36)] -Aggregate Attributes [3]: [sum(sales#34)#126, sum(returns#35)#127, sum(profit#36)#128] -Results [5]: [channel#111, id#112, sum(sales#34)#126 AS sales#129, sum(returns#35)#127 AS returns#130, sum(profit#36)#128 AS profit#131] - -(107) TakeOrderedAndProject -Input [5]: [channel#111, id#112, sales#129, returns#130, profit#131] -Arguments: 100, [channel#111 ASC NULLS FIRST, id#112 ASC NULLS FIRST], [channel#111, id#112, sales#129, returns#130, profit#131] +(83) CometFilter +Input [2]: [web_site_sk#80, web_site_id#81] +Condition : isnotnull(web_site_sk#80) + +(84) CometBroadcastExchange +Input [2]: [web_site_sk#80, web_site_id#81] +Arguments: [web_site_sk#80, web_site_id#81] + +(85) CometBroadcastHashJoin +Left output [7]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77] +Right output [2]: [web_site_sk#80, web_site_id#81] +Arguments: [ws_web_site_sk#67], [web_site_sk#80], Inner, BuildRight + +(86) CometProject +Input [9]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_sk#80, web_site_id#81] +Arguments: [ws_item_sk#66, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81], [ws_item_sk#66, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81] + +(87) ReusedExchange [Reuses operator id: 26] +Output [1]: [i_item_sk#82] + +(88) CometBroadcastHashJoin +Left output [7]: [ws_item_sk#66, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81] +Right output [1]: [i_item_sk#82] +Arguments: [ws_item_sk#66], [i_item_sk#82], Inner, BuildRight + +(89) CometProject +Input [8]: [ws_item_sk#66, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81, i_item_sk#82] +Arguments: [ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81], [ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81] + +(90) ReusedExchange [Reuses operator id: 32] +Output [1]: [p_promo_sk#83] + +(91) CometBroadcastHashJoin +Left output [6]: [ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81] +Right output [1]: [p_promo_sk#83] +Arguments: [ws_promo_sk#68], [p_promo_sk#83], Inner, BuildRight + +(92) CometProject +Input [7]: [ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81, p_promo_sk#83] +Arguments: [ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81], [ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81] + +(93) CometHashAggregate +Input [5]: [ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81] +Keys [1]: [web_site_id#81] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(coalesce(cast(wr_return_amt#76 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#71 - coalesce(cast(wr_net_loss#77 as decimal(12,2)), 0.00)))] + +(94) CometColumnarExchange +Input [6]: [web_site_id#81, sum#84, sum#85, isEmpty#86, sum#87, isEmpty#88] +Arguments: hashpartitioning(web_site_id#81, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(95) ColumnarToRow [codegen id : 3] +Input [6]: [web_site_id#81, sum#84, sum#85, isEmpty#86, sum#87, isEmpty#88] + +(96) HashAggregate [codegen id : 3] +Input [6]: [web_site_id#81, sum#84, sum#85, isEmpty#86, sum#87, isEmpty#88] +Keys [1]: [web_site_id#81] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#70)), sum(coalesce(cast(wr_return_amt#76 as decimal(12,2)), 0.00)), sum((ws_net_profit#71 - coalesce(cast(wr_net_loss#77 as decimal(12,2)), 0.00)))] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#70))#89, sum(coalesce(cast(wr_return_amt#76 as decimal(12,2)), 0.00))#90, sum((ws_net_profit#71 - coalesce(cast(wr_net_loss#77 as decimal(12,2)), 0.00)))#91] +Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#70))#89,17,2) AS sales#92, sum(coalesce(cast(wr_return_amt#76 as decimal(12,2)), 0.00))#90 AS returns#93, sum((ws_net_profit#71 - coalesce(cast(wr_net_loss#77 as decimal(12,2)), 0.00)))#91 AS profit#94, web channel AS channel#95, concat(web_site, web_site_id#81) AS id#96] + +(97) Union + +(98) Expand [codegen id : 4] +Input [5]: [sales#30, returns#31, profit#32, channel#33, id#34] +Arguments: [[sales#30, returns#31, profit#32, channel#33, id#34, 0], [sales#30, returns#31, profit#32, channel#33, null, 1], [sales#30, returns#31, profit#32, null, null, 3]], [sales#30, returns#31, profit#32, channel#97, id#98, spark_grouping_id#99] + +(99) HashAggregate [codegen id : 4] +Input [6]: [sales#30, returns#31, profit#32, channel#97, id#98, spark_grouping_id#99] +Keys [3]: [channel#97, id#98, spark_grouping_id#99] +Functions [3]: [partial_sum(sales#30), partial_sum(returns#31), partial_sum(profit#32)] +Aggregate Attributes [6]: [sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] +Results [9]: [channel#97, id#98, spark_grouping_id#99, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] + +(100) RowToColumnar +Input [9]: [channel#97, id#98, spark_grouping_id#99, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] + +(101) CometColumnarExchange +Input [9]: [channel#97, id#98, spark_grouping_id#99, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] +Arguments: hashpartitioning(channel#97, id#98, spark_grouping_id#99, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(102) ColumnarToRow [codegen id : 5] +Input [9]: [channel#97, id#98, spark_grouping_id#99, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] + +(103) HashAggregate [codegen id : 5] +Input [9]: [channel#97, id#98, spark_grouping_id#99, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] +Keys [3]: [channel#97, id#98, spark_grouping_id#99] +Functions [3]: [sum(sales#30), sum(returns#31), sum(profit#32)] +Aggregate Attributes [3]: [sum(sales#30)#112, sum(returns#31)#113, sum(profit#32)#114] +Results [5]: [channel#97, id#98, sum(sales#30)#112 AS sales#115, sum(returns#31)#113 AS returns#116, sum(profit#32)#114 AS profit#117] + +(104) TakeOrderedAndProject +Input [5]: [channel#97, id#98, sales#115, returns#116, profit#117] +Arguments: 100, [channel#97 ASC NULLS FIRST, id#98 ASC NULLS FIRST], [channel#97, id#98, sales#115, returns#116, profit#117] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (112) -+- * ColumnarToRow (111) - +- CometProject (110) - +- CometFilter (109) - +- CometScan parquet spark_catalog.default.date_dim (108) +BroadcastExchange (109) ++- * ColumnarToRow (108) + +- CometProject (107) + +- CometFilter (106) + +- CometScan parquet spark_catalog.default.date_dim (105) -(108) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#132] +(105) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#15] 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-22), IsNotNull(d_date_sk)] ReadSchema: struct -(109) CometFilter -Input [2]: [d_date_sk#14, d_date#132] -Condition : (((isnotnull(d_date#132) AND (d_date#132 >= 2000-08-23)) AND (d_date#132 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) +(106) CometFilter +Input [2]: [d_date_sk#14, d_date#15] +Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#15 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) -(110) CometProject -Input [2]: [d_date_sk#14, d_date#132] +(107) CometProject +Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(111) ColumnarToRow [codegen id : 1] +(108) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(112) BroadcastExchange +(109) BroadcastExchange Input [1]: [d_date_sk#14] -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=11] -Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 71 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#72 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/simplified.txt index cdc8dc486..dfcc35f6e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/simplified.txt @@ -1,182 +1,124 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (32) + WholeStageCodegen (5) 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 (31) - 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 (10) - HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [s_store_id] #2 - WholeStageCodegen (9) - HashAggregate [s_store_id,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - SortMergeJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - WholeStageCodegen (2) - Sort [ss_item_sk,ss_ticket_number] - InputAdapter - Exchange [ss_item_sk,ss_ticket_number] #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - WholeStageCodegen (4) - Sort [sr_item_sk,sr_ticket_number] - InputAdapter - Exchange [sr_item_sk,sr_ticket_number] #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - 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 (7) - ColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - WholeStageCodegen (20) - HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [cp_catalog_page_id] #9 - WholeStageCodegen (19) - HashAggregate [cp_catalog_page_id,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_catalog_page_sk,cp_catalog_page_sk] - Project [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] - InputAdapter - WholeStageCodegen (12) - Sort [cs_item_sk,cs_order_number] - InputAdapter - Exchange [cs_item_sk,cs_order_number] #10 - WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - WholeStageCodegen (14) - Sort [cr_item_sk,cr_order_number] - InputAdapter - Exchange [cr_item_sk,cr_order_number] #11 - WholeStageCodegen (13) - ColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - InputAdapter - ReusedExchange [i_item_sk] #7 - InputAdapter - ReusedExchange [p_promo_sk] #8 - WholeStageCodegen (30) - HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [web_site_id] #13 - WholeStageCodegen (29) - HashAggregate [web_site_id,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_promo_sk,p_promo_sk] - Project [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - SortMergeJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] - InputAdapter - WholeStageCodegen (22) - Sort [ws_item_sk,ws_order_number] - InputAdapter - Exchange [ws_item_sk,ws_order_number] #14 - WholeStageCodegen (21) - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - WholeStageCodegen (24) - Sort [wr_item_sk,wr_order_number] - InputAdapter - Exchange [wr_item_sk,wr_order_number] #15 - WholeStageCodegen (23) - ColumnarToRow - InputAdapter - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #16 - WholeStageCodegen (26) - ColumnarToRow - InputAdapter - CometFilter [web_site_sk,web_site_id] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - InputAdapter - ReusedExchange [i_item_sk] #7 - InputAdapter - ReusedExchange [p_promo_sk] #8 + ColumnarToRow + InputAdapter + CometColumnarExchange [channel,id,spark_grouping_id] #1 + RowToColumnar + WholeStageCodegen (4) + 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 (1) + HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [s_store_id] #2 + CometHashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] + CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] + CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] + CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] + CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk,ss_ticket_number] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometColumnarExchange [sr_item_sk,sr_ticket_number] #5 + CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [s_store_sk,s_store_id] #7 + CometFilter [s_store_sk,s_store_id] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange [i_item_sk] #8 + CometProject [i_item_sk] + CometFilter [i_item_sk,i_current_price] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] + CometBroadcastExchange [p_promo_sk] #9 + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] + CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] + WholeStageCodegen (2) + HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [cp_catalog_page_id] #10 + CometHashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] + CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] + CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] + CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] + CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometColumnarExchange [cs_item_sk,cs_order_number] #11 + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometColumnarExchange [cr_item_sk,cr_order_number] #12 + CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #13 + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + ReusedExchange [i_item_sk] #8 + ReusedExchange [p_promo_sk] #9 + WholeStageCodegen (3) + HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [web_site_id] #14 + CometHashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] + CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] + CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] + CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] + CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometColumnarExchange [ws_item_sk,ws_order_number] #15 + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometColumnarExchange [wr_item_sk,wr_order_number] #16 + CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + CometBroadcastExchange [web_site_sk,web_site_id] #17 + CometFilter [web_site_sk,web_site_id] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + ReusedExchange [i_item_sk] #8 + ReusedExchange [p_promo_sk] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/explain.txt index 3c9a67f78..2223ccb22 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/explain.txt @@ -1,54 +1,51 @@ == Physical Plan == -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) +* ColumnarToRow (47) ++- CometTakeOrderedAndProject (46) + +- CometProject (45) + +- CometBroadcastHashJoin (44) + :- CometProject (40) + : +- CometBroadcastHashJoin (39) + : :- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometFilter (17) + : : : +- CometHashAggregate (16) + : : : +- CometColumnarExchange (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) + : : +- CometBroadcastExchange (33) + : : +- CometFilter (32) + : : +- CometHashAggregate (31) + : : +- CometColumnarExchange (30) + : : +- CometHashAggregate (29) + : : +- CometHashAggregate (28) + : : +- CometColumnarExchange (27) + : : +- CometHashAggregate (26) + : : +- CometProject (25) + : : +- CometBroadcastHashJoin (24) + : : :- CometProject (22) + : : : +- CometBroadcastHashJoin (21) + : : : :- CometFilter (19) + : : : : +- CometScan parquet spark_catalog.default.catalog_returns (18) + : : : +- ReusedExchange (20) + : : +- ReusedExchange (23) + : +- CometBroadcastExchange (38) + : +- CometFilter (37) + : +- CometScan parquet spark_catalog.default.customer (36) + +- CometBroadcastExchange (43) + +- CometFilter (42) + +- CometScan parquet spark_catalog.default.customer_address (41) (1) Scan parquet spark_catalog.default.catalog_returns @@ -120,207 +117,187 @@ 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))] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(17) HashAggregate [codegen id : 7] +(16) CometHashAggregate 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#9 AS ctr_state#13, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#11,17,2) AS ctr_total_return#14] -(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) CometFilter +Input [3]: [ctr_customer_sk#11, ctr_state#12, ctr_total_return#13] +Condition : isnotnull(ctr_total_return#13) -(19) 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] +(18) Scan parquet spark_catalog.default.catalog_returns +Output [4]: [cr_returning_customer_sk#14, cr_returning_addr_sk#15, cr_return_amt_inc_tax#16, cr_returned_date_sk#17] 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#17), dynamicpruningexpression(cr_returned_date_sk#17 IN dynamicpruning#18)] PushedFilters: [IsNotNull(cr_returning_addr_sk)] ReadSchema: struct -(20) 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) - -(21) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#20] - -(22) CometBroadcastHashJoin -Left output [4]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, cr_returned_date_sk#18] -Right output [1]: [d_date_sk#20] -Arguments: [cr_returned_date_sk#18], [d_date_sk#20], Inner, BuildRight - -(23) CometProject -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] -Arguments: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17], [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17] - -(24) ReusedExchange [Reuses operator id: 11] -Output [2]: [ca_address_sk#21, ca_state#22] - -(25) CometBroadcastHashJoin -Left output [3]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17] -Right output [2]: [ca_address_sk#21, ca_state#22] -Arguments: [cr_returning_addr_sk#16], [ca_address_sk#21], Inner, BuildRight - -(26) CometProject -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] -Arguments: [cr_returning_customer_sk#15, cr_return_amt_inc_tax#17, ca_state#22], [cr_returning_customer_sk#15, cr_return_amt_inc_tax#17, ca_state#22] - -(27) CometHashAggregate -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))] - -(28) ColumnarToRow [codegen id : 2] -Input [3]: [cr_returning_customer_sk#15, ca_state#22, sum#23] - -(29) Exchange -Input [3]: [cr_returning_customer_sk#15, ca_state#22, sum#23] -Arguments: hashpartitioning(cr_returning_customer_sk#15, ca_state#22, 5), ENSURE_REQUIREMENTS, [plan_id=2] - -(30) HashAggregate [codegen id : 3] -Input [3]: [cr_returning_customer_sk#15, ca_state#22, sum#23] -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#24, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#17))#11,17,2) AS ctr_total_return#25] - -(31) HashAggregate [codegen id : 3] -Input [2]: [ctr_state#24, ctr_total_return#25] -Keys [1]: [ctr_state#24] -Functions [1]: [partial_avg(ctr_total_return#25)] -Aggregate Attributes [2]: [sum#26, count#27] -Results [3]: [ctr_state#24, sum#28, count#29] - -(32) Exchange -Input [3]: [ctr_state#24, sum#28, count#29] -Arguments: hashpartitioning(ctr_state#24, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(33) HashAggregate [codegen id : 4] -Input [3]: [ctr_state#24, sum#28, count#29] -Keys [1]: [ctr_state#24] -Functions [1]: [avg(ctr_total_return#25)] -Aggregate Attributes [1]: [avg(ctr_total_return#25)#30] -Results [2]: [(avg(ctr_total_return#25)#30 * 1.2) AS (avg(ctr_total_return) * 1.2)#31, ctr_state#24] - -(34) Filter [codegen id : 4] -Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_state#24] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#31) - -(35) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_state#24] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=4] - -(36) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ctr_state#13] -Right keys [1]: [ctr_state#24] -Join type: Inner -Join condition: (cast(ctr_total_return#14 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#31) - -(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)#31, ctr_state#24] - -(38) Scan parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] +(19) CometFilter +Input [4]: [cr_returning_customer_sk#14, cr_returning_addr_sk#15, cr_return_amt_inc_tax#16, cr_returned_date_sk#17] +Condition : isnotnull(cr_returning_addr_sk#15) + +(20) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#19] + +(21) CometBroadcastHashJoin +Left output [4]: [cr_returning_customer_sk#14, cr_returning_addr_sk#15, cr_return_amt_inc_tax#16, cr_returned_date_sk#17] +Right output [1]: [d_date_sk#19] +Arguments: [cr_returned_date_sk#17], [d_date_sk#19], Inner, BuildRight + +(22) CometProject +Input [5]: [cr_returning_customer_sk#14, cr_returning_addr_sk#15, cr_return_amt_inc_tax#16, cr_returned_date_sk#17, d_date_sk#19] +Arguments: [cr_returning_customer_sk#14, cr_returning_addr_sk#15, cr_return_amt_inc_tax#16], [cr_returning_customer_sk#14, cr_returning_addr_sk#15, cr_return_amt_inc_tax#16] + +(23) ReusedExchange [Reuses operator id: 11] +Output [2]: [ca_address_sk#20, ca_state#21] + +(24) CometBroadcastHashJoin +Left output [3]: [cr_returning_customer_sk#14, cr_returning_addr_sk#15, cr_return_amt_inc_tax#16] +Right output [2]: [ca_address_sk#20, ca_state#21] +Arguments: [cr_returning_addr_sk#15], [ca_address_sk#20], Inner, BuildRight + +(25) CometProject +Input [5]: [cr_returning_customer_sk#14, cr_returning_addr_sk#15, cr_return_amt_inc_tax#16, ca_address_sk#20, ca_state#21] +Arguments: [cr_returning_customer_sk#14, cr_return_amt_inc_tax#16, ca_state#21], [cr_returning_customer_sk#14, cr_return_amt_inc_tax#16, ca_state#21] + +(26) CometHashAggregate +Input [3]: [cr_returning_customer_sk#14, cr_return_amt_inc_tax#16, ca_state#21] +Keys [2]: [cr_returning_customer_sk#14, ca_state#21] +Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#16))] + +(27) CometColumnarExchange +Input [3]: [cr_returning_customer_sk#14, ca_state#21, sum#22] +Arguments: hashpartitioning(cr_returning_customer_sk#14, ca_state#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(28) CometHashAggregate +Input [3]: [cr_returning_customer_sk#14, ca_state#21, sum#22] +Keys [2]: [cr_returning_customer_sk#14, ca_state#21] +Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#16))] + +(29) CometHashAggregate +Input [2]: [ctr_state#23, ctr_total_return#24] +Keys [1]: [ctr_state#23] +Functions [1]: [partial_avg(ctr_total_return#24)] + +(30) CometColumnarExchange +Input [3]: [ctr_state#23, sum#25, count#26] +Arguments: hashpartitioning(ctr_state#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(31) CometHashAggregate +Input [3]: [ctr_state#23, sum#25, count#26] +Keys [1]: [ctr_state#23] +Functions [1]: [avg(ctr_total_return#24)] + +(32) CometFilter +Input [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_state#23] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#27) + +(33) CometBroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_state#23] +Arguments: [(avg(ctr_total_return) * 1.2)#27, ctr_state#23] + +(34) CometBroadcastHashJoin +Left output [3]: [ctr_customer_sk#11, ctr_state#12, ctr_total_return#13] +Right output [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_state#23] +Arguments: [ctr_state#12], [ctr_state#23], Inner, (cast(ctr_total_return#13 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#27), BuildRight + +(35) CometProject +Input [5]: [ctr_customer_sk#11, ctr_state#12, ctr_total_return#13, (avg(ctr_total_return) * 1.2)#27, ctr_state#23] +Arguments: [ctr_customer_sk#11, ctr_total_return#13], [ctr_customer_sk#11, ctr_total_return#13] + +(36) Scan parquet spark_catalog.default.customer +Output [6]: [c_customer_sk#28, c_customer_id#29, c_current_addr_sk#30, c_salutation#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_current_addr_sk)] ReadSchema: struct -(39) CometFilter -Input [6]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] -Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_current_addr_sk#34)) - -(40) ColumnarToRow [codegen id : 5] -Input [6]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] +(37) CometFilter +Input [6]: [c_customer_sk#28, c_customer_id#29, c_current_addr_sk#30, c_salutation#31, c_first_name#32, c_last_name#33] +Condition : (isnotnull(c_customer_sk#28) AND isnotnull(c_current_addr_sk#30)) -(41) BroadcastExchange -Input [6]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +(38) CometBroadcastExchange +Input [6]: [c_customer_sk#28, c_customer_id#29, c_current_addr_sk#30, c_salutation#31, c_first_name#32, c_last_name#33] +Arguments: [c_customer_sk#28, c_customer_id#29, c_current_addr_sk#30, c_salutation#31, c_first_name#32, c_last_name#33] -(42) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ctr_customer_sk#12] -Right keys [1]: [c_customer_sk#32] -Join type: Inner -Join condition: None +(39) CometBroadcastHashJoin +Left output [2]: [ctr_customer_sk#11, ctr_total_return#13] +Right output [6]: [c_customer_sk#28, c_customer_id#29, c_current_addr_sk#30, c_salutation#31, c_first_name#32, c_last_name#33] +Arguments: [ctr_customer_sk#11], [c_customer_sk#28], Inner, BuildRight -(43) Project [codegen id : 7] -Output [6]: [ctr_total_return#14, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] -Input [8]: [ctr_customer_sk#12, ctr_total_return#14, c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] +(40) CometProject +Input [8]: [ctr_customer_sk#11, ctr_total_return#13, c_customer_sk#28, c_customer_id#29, c_current_addr_sk#30, c_salutation#31, c_first_name#32, c_last_name#33] +Arguments: [ctr_total_return#13, c_customer_id#29, c_current_addr_sk#30, c_salutation#31, c_first_name#32, c_last_name#33], [ctr_total_return#13, c_customer_id#29, c_current_addr_sk#30, c_salutation#31, c_first_name#32, c_last_name#33] -(44) Scan parquet spark_catalog.default.customer_address -Output [12]: [ca_address_sk#38, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#45, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] +(41) Scan parquet spark_catalog.default.customer_address +Output [12]: [ca_address_sk#34, ca_street_number#35, ca_street_name#36, ca_street_type#37, ca_suite_number#38, ca_city#39, ca_county#40, ca_state#41, ca_zip#42, ca_country#43, ca_gmt_offset#44, ca_location_type#45] 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 -(45) CometFilter -Input [12]: [ca_address_sk#38, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#45, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] -Condition : ((isnotnull(ca_state#45) AND (ca_state#45 = GA)) AND isnotnull(ca_address_sk#38)) +(42) CometFilter +Input [12]: [ca_address_sk#34, ca_street_number#35, ca_street_name#36, ca_street_type#37, ca_suite_number#38, ca_city#39, ca_county#40, ca_state#41, ca_zip#42, ca_country#43, ca_gmt_offset#44, ca_location_type#45] +Condition : ((isnotnull(ca_state#41) AND (ca_state#41 = GA)) AND isnotnull(ca_address_sk#34)) -(46) ColumnarToRow [codegen id : 6] -Input [12]: [ca_address_sk#38, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#45, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] +(43) CometBroadcastExchange +Input [12]: [ca_address_sk#34, ca_street_number#35, ca_street_name#36, ca_street_type#37, ca_suite_number#38, ca_city#39, ca_county#40, ca_state#41, ca_zip#42, ca_country#43, ca_gmt_offset#44, ca_location_type#45] +Arguments: [ca_address_sk#34, ca_street_number#35, ca_street_name#36, ca_street_type#37, ca_suite_number#38, ca_city#39, ca_county#40, ca_state#41, ca_zip#42, ca_country#43, ca_gmt_offset#44, ca_location_type#45] -(47) BroadcastExchange -Input [12]: [ca_address_sk#38, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#45, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +(44) CometBroadcastHashJoin +Left output [6]: [ctr_total_return#13, c_customer_id#29, c_current_addr_sk#30, c_salutation#31, c_first_name#32, c_last_name#33] +Right output [12]: [ca_address_sk#34, ca_street_number#35, ca_street_name#36, ca_street_type#37, ca_suite_number#38, ca_city#39, ca_county#40, ca_state#41, ca_zip#42, ca_country#43, ca_gmt_offset#44, ca_location_type#45] +Arguments: [c_current_addr_sk#30], [ca_address_sk#34], Inner, BuildRight -(48) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#34] -Right keys [1]: [ca_address_sk#38] -Join type: Inner -Join condition: None +(45) CometProject +Input [18]: [ctr_total_return#13, c_customer_id#29, c_current_addr_sk#30, c_salutation#31, c_first_name#32, c_last_name#33, ca_address_sk#34, ca_street_number#35, ca_street_name#36, ca_street_type#37, ca_suite_number#38, ca_city#39, ca_county#40, ca_state#41, ca_zip#42, ca_country#43, ca_gmt_offset#44, ca_location_type#45] +Arguments: [c_customer_id#29, c_salutation#31, c_first_name#32, c_last_name#33, ca_street_number#35, ca_street_name#36, ca_street_type#37, ca_suite_number#38, ca_city#39, ca_county#40, ca_state#41, ca_zip#42, ca_country#43, ca_gmt_offset#44, ca_location_type#45, ctr_total_return#13], [c_customer_id#29, c_salutation#31, c_first_name#32, c_last_name#33, ca_street_number#35, ca_street_name#36, ca_street_type#37, ca_suite_number#38, ca_city#39, ca_county#40, ca_state#41, ca_zip#42, ca_country#43, ca_gmt_offset#44, ca_location_type#45, ctr_total_return#13] -(49) Project [codegen id : 7] -Output [16]: [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#45, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49, ctr_total_return#14] -Input [18]: [ctr_total_return#14, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, ca_address_sk#38, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#45, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] +(46) CometTakeOrderedAndProject +Input [16]: [c_customer_id#29, c_salutation#31, c_first_name#32, c_last_name#33, ca_street_number#35, ca_street_name#36, ca_street_type#37, ca_suite_number#38, ca_city#39, ca_county#40, ca_state#41, ca_zip#42, ca_country#43, ca_gmt_offset#44, ca_location_type#45, ctr_total_return#13] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#29 ASC NULLS FIRST,c_salutation#31 ASC NULLS FIRST,c_first_name#32 ASC NULLS FIRST,c_last_name#33 ASC NULLS FIRST,ca_street_number#35 ASC NULLS FIRST,ca_street_name#36 ASC NULLS FIRST,ca_street_type#37 ASC NULLS FIRST,ca_suite_number#38 ASC NULLS FIRST,ca_city#39 ASC NULLS FIRST,ca_county#40 ASC NULLS FIRST,ca_state#41 ASC NULLS FIRST,ca_zip#42 ASC NULLS FIRST,ca_country#43 ASC NULLS FIRST,ca_gmt_offset#44 ASC NULLS FIRST,ca_location_type#45 ASC NULLS FIRST,ctr_total_return#13 ASC NULLS FIRST], output=[c_customer_id#29,c_salutation#31,c_first_name#32,c_last_name#33,ca_street_number#35,ca_street_name#36,ca_street_type#37,ca_suite_number#38,ca_city#39,ca_county#40,ca_state#41,ca_zip#42,ca_country#43,ca_gmt_offset#44,ca_location_type#45,ctr_total_return#13]), [c_customer_id#29, c_salutation#31, c_first_name#32, c_last_name#33, ca_street_number#35, ca_street_name#36, ca_street_type#37, ca_suite_number#38, ca_city#39, ca_county#40, ca_state#41, ca_zip#42, ca_country#43, ca_gmt_offset#44, ca_location_type#45, ctr_total_return#13], 100, [c_customer_id#29 ASC NULLS FIRST, c_salutation#31 ASC NULLS FIRST, c_first_name#32 ASC NULLS FIRST, c_last_name#33 ASC NULLS FIRST, ca_street_number#35 ASC NULLS FIRST, ca_street_name#36 ASC NULLS FIRST, ca_street_type#37 ASC NULLS FIRST, ca_suite_number#38 ASC NULLS FIRST, ca_city#39 ASC NULLS FIRST, ca_county#40 ASC NULLS FIRST, ca_state#41 ASC NULLS FIRST, ca_zip#42 ASC NULLS FIRST, ca_country#43 ASC NULLS FIRST, ca_gmt_offset#44 ASC NULLS FIRST, ca_location_type#45 ASC NULLS FIRST, ctr_total_return#13 ASC NULLS FIRST], [c_customer_id#29, c_salutation#31, c_first_name#32, c_last_name#33, ca_street_number#35, ca_street_name#36, ca_street_type#37, ca_suite_number#38, ca_city#39, ca_county#40, ca_state#41, ca_zip#42, ca_country#43, ca_gmt_offset#44, ca_location_type#45, ctr_total_return#13] -(50) TakeOrderedAndProject -Input [16]: [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#45, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49, ctr_total_return#14] -Arguments: 100, [c_customer_id#33 ASC NULLS FIRST, c_salutation#35 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, c_last_name#37 ASC NULLS FIRST, ca_street_number#39 ASC NULLS FIRST, ca_street_name#40 ASC NULLS FIRST, ca_street_type#41 ASC NULLS FIRST, ca_suite_number#42 ASC NULLS FIRST, ca_city#43 ASC NULLS FIRST, ca_county#44 ASC NULLS FIRST, ca_state#45 ASC NULLS FIRST, ca_zip#46 ASC NULLS FIRST, ca_country#47 ASC NULLS FIRST, ca_gmt_offset#48 ASC NULLS FIRST, ca_location_type#49 ASC NULLS FIRST, ctr_total_return#14 ASC NULLS FIRST], [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#45, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49, ctr_total_return#14] +(47) ColumnarToRow [codegen id : 1] +Input [16]: [c_customer_id#29, c_salutation#31, c_first_name#32, c_last_name#33, ca_street_number#35, ca_street_name#36, ca_street_type#37, ca_suite_number#38, ca_city#39, ca_county#40, ca_state#41, ca_zip#42, ca_country#43, ca_gmt_offset#44, ca_location_type#45, ctr_total_return#13] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (55) -+- * ColumnarToRow (54) - +- CometProject (53) - +- CometFilter (52) - +- CometScan parquet spark_catalog.default.date_dim (51) +BroadcastExchange (52) ++- * ColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) -(51) Scan parquet spark_catalog.default.date_dim +(48) 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 -(52) CometFilter +(49) 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)) -(53) CometProject +(50) CometProject Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(54) ColumnarToRow [codegen id : 1] +(51) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(55) BroadcastExchange +(52) BroadcastExchange Input [1]: [d_date_sk#6] -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=4] -Subquery:2 Hosting operator id = 19 Hosting Expression = cr_returned_date_sk#18 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 18 Hosting Expression = cr_returned_date_sk#17 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/simplified.txt index 11f3a0ff1..feaeefa9a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/simplified.txt @@ -1,76 +1,58 @@ -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 (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] - BroadcastHashJoin [ctr_customer_sk,c_customer_sk] - Project [ctr_customer_sk,ctr_total_return] - BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] - Filter [ctr_total_return] - HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_customer_sk,ctr_state,ctr_total_return,sum] - InputAdapter - Exchange [cr_returning_customer_sk,ca_state] #1 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [cr_returning_customer_sk,ca_state,sum,cr_return_amt_inc_tax] - CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] - CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] - CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [ca_address_sk,ca_state] #4 - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - 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] #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] #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometHashAggregate [cr_returning_customer_sk,ca_state,sum,cr_return_amt_inc_tax] - CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] - CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] - CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_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 - ReusedExchange [d_date_sk] #3 - ReusedExchange [ca_address_sk,ca_state] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - 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 #9 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [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] + CometProject [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] + CometBroadcastHashJoin [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] + CometProject [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + CometBroadcastHashJoin [ctr_customer_sk,ctr_total_return,c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + CometProject [ctr_customer_sk,ctr_total_return] + CometBroadcastHashJoin [ctr_customer_sk,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_state] + CometFilter [ctr_customer_sk,ctr_state,ctr_total_return] + CometHashAggregate [ctr_customer_sk,ctr_state,ctr_total_return,cr_returning_customer_sk,ca_state,sum,sum(UnscaledValue(cr_return_amt_inc_tax))] + CometColumnarExchange [cr_returning_customer_sk,ca_state] #1 + CometHashAggregate [cr_returning_customer_sk,ca_state,sum,cr_return_amt_inc_tax] + CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] + CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] + CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [ca_address_sk,ca_state] #4 + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_state] #5 + CometFilter [(avg(ctr_total_return) * 1.2),ctr_state] + CometHashAggregate [(avg(ctr_total_return) * 1.2),ctr_state,sum,count,avg(ctr_total_return)] + CometColumnarExchange [ctr_state] #6 + CometHashAggregate [ctr_state,sum,count,ctr_total_return] + CometHashAggregate [ctr_state,ctr_total_return,cr_returning_customer_sk,ca_state,sum,sum(UnscaledValue(cr_return_amt_inc_tax))] + CometColumnarExchange [cr_returning_customer_sk,ca_state] #7 + CometHashAggregate [cr_returning_customer_sk,ca_state,sum,cr_return_amt_inc_tax] + CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] + CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] + CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_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 + ReusedExchange [d_date_sk] #3 + ReusedExchange [ca_address_sk,ca_state] #4 + CometBroadcastExchange [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] #8 + CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] #9 + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/explain.txt index 00e38f694..2519fbc97 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (26) -+- * HashAggregate (25) - +- Exchange (24) - +- * ColumnarToRow (23) +* ColumnarToRow (26) ++- CometTakeOrderedAndProject (25) + +- CometHashAggregate (24) + +- CometColumnarExchange (23) +- CometHashAggregate (22) +- CometProject (21) +- CometBroadcastHashJoin (20) @@ -132,23 +132,21 @@ 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: [] -(23) ColumnarToRow [codegen id : 1] +(23) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(24) 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] -(26) TakeOrderedAndProject +(25) CometTakeOrderedAndProject +Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#2 ASC NULLS FIRST], output=[i_item_id#2,i_item_desc#3,i_current_price#4]), [i_item_id#2, i_item_desc#3, i_current_price#4], 100, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4] + +(26) ColumnarToRow [codegen id : 1] 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 ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/simplified.txt index 71a1c8e17..869073de1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/simplified.txt @@ -1,38 +1,36 @@ -TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - 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 (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,i_item_id,i_item_desc,i_current_price,ss_item_sk] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2 - CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,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,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] - CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 - CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_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_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometProject [ss_item_sk] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] + CometHashAggregate [i_item_id,i_item_desc,i_current_price] + CometColumnarExchange [i_item_id,i_item_desc,i_current_price] #1 + CometHashAggregate [i_item_id,i_item_desc,i_current_price] + CometProject [i_item_id,i_item_desc,i_current_price] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,ss_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2 + CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,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,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] + CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] + CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 + CometProject [inv_item_sk,inv_date_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_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_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [ss_item_sk] + CometFilter [ss_item_sk,ss_sold_date_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-spark3_5/q83/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/explain.txt index 1838b9a77..972a58976 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/explain.txt @@ -1,54 +1,52 @@ == Physical Plan == -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) +* ColumnarToRow (48) ++- CometTakeOrderedAndProject (47) + +- CometProject (46) + +- CometBroadcastHashJoin (45) + :- CometProject (32) + : +- CometBroadcastHashJoin (31) + : :- CometHashAggregate (18) + : : +- CometColumnarExchange (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) + : +- CometBroadcastExchange (30) + : +- CometHashAggregate (29) + : +- CometColumnarExchange (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) + +- CometBroadcastExchange (44) + +- CometHashAggregate (43) + +- CometColumnarExchange (42) + +- CometHashAggregate (41) + +- CometProject (40) + +- CometBroadcastHashJoin (39) + :- CometProject (37) + : +- CometBroadcastHashJoin (36) + : :- CometFilter (34) + : : +- CometScan parquet spark_catalog.default.web_returns (33) + : +- ReusedExchange (35) + +- ReusedExchange (38) (1) Scan parquet spark_catalog.default.store_returns @@ -98,7 +96,7 @@ ReadSchema: struct Input [2]: [d_date_sk#7, d_date#8] Condition : isnotnull(d_date_sk#7) -(10) ReusedExchange [Reuses operator id: 60] +(10) ReusedExchange [Reuses operator id: 58] Output [1]: [d_date#9] (11) CometBroadcastHashJoin @@ -128,247 +126,233 @@ Input [2]: [sr_return_quantity#2, i_item_id#6] Keys [1]: [i_item_id#6] Functions [1]: [partial_sum(sr_return_quantity#2)] -(17) ColumnarToRow [codegen id : 1] +(17) CometColumnarExchange Input [2]: [i_item_id#6, sum#10] +Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(18) Exchange -Input [2]: [i_item_id#6, sum#10] -Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [plan_id=1] - -(19) HashAggregate [codegen id : 6] +(18) CometHashAggregate 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)#11] -Results [2]: [i_item_id#6 AS item_id#12, sum(sr_return_quantity#2)#11 AS sr_item_qty#13] -(20) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#14, cr_return_quantity#15, cr_returned_date_sk#16] +(19) Scan parquet spark_catalog.default.catalog_returns +Output [3]: [cr_item_sk#11, cr_return_quantity#12, cr_returned_date_sk#13] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#16), dynamicpruningexpression(cr_returned_date_sk#16 IN dynamicpruning#17)] +PartitionFilters: [isnotnull(cr_returned_date_sk#13), dynamicpruningexpression(cr_returned_date_sk#13 IN dynamicpruning#14)] PushedFilters: [IsNotNull(cr_item_sk)] ReadSchema: struct -(21) CometFilter -Input [3]: [cr_item_sk#14, cr_return_quantity#15, cr_returned_date_sk#16] -Condition : isnotnull(cr_item_sk#14) - -(22) ReusedExchange [Reuses operator id: 5] -Output [2]: [i_item_sk#18, i_item_id#19] - -(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 - -(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] - -(25) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#20] - -(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 - -(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)] - -(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=3] - -(33) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [item_id#12] -Right keys [1]: [item_id#23] -Join type: Inner -Join condition: None - -(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] - -(35) Scan parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] +(20) CometFilter +Input [3]: [cr_item_sk#11, cr_return_quantity#12, cr_returned_date_sk#13] +Condition : isnotnull(cr_item_sk#11) + +(21) ReusedExchange [Reuses operator id: 5] +Output [2]: [i_item_sk#15, i_item_id#16] + +(22) CometBroadcastHashJoin +Left output [3]: [cr_item_sk#11, cr_return_quantity#12, cr_returned_date_sk#13] +Right output [2]: [i_item_sk#15, i_item_id#16] +Arguments: [cr_item_sk#11], [i_item_sk#15], Inner, BuildRight + +(23) CometProject +Input [5]: [cr_item_sk#11, cr_return_quantity#12, cr_returned_date_sk#13, i_item_sk#15, i_item_id#16] +Arguments: [cr_return_quantity#12, cr_returned_date_sk#13, i_item_id#16], [cr_return_quantity#12, cr_returned_date_sk#13, i_item_id#16] + +(24) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_date_sk#17] + +(25) CometBroadcastHashJoin +Left output [3]: [cr_return_quantity#12, cr_returned_date_sk#13, i_item_id#16] +Right output [1]: [d_date_sk#17] +Arguments: [cr_returned_date_sk#13], [d_date_sk#17], Inner, BuildRight + +(26) CometProject +Input [4]: [cr_return_quantity#12, cr_returned_date_sk#13, i_item_id#16, d_date_sk#17] +Arguments: [cr_return_quantity#12, i_item_id#16], [cr_return_quantity#12, i_item_id#16] + +(27) CometHashAggregate +Input [2]: [cr_return_quantity#12, i_item_id#16] +Keys [1]: [i_item_id#16] +Functions [1]: [partial_sum(cr_return_quantity#12)] + +(28) CometColumnarExchange +Input [2]: [i_item_id#16, sum#18] +Arguments: hashpartitioning(i_item_id#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(29) CometHashAggregate +Input [2]: [i_item_id#16, sum#18] +Keys [1]: [i_item_id#16] +Functions [1]: [sum(cr_return_quantity#12)] + +(30) CometBroadcastExchange +Input [2]: [item_id#19, cr_item_qty#20] +Arguments: [item_id#19, cr_item_qty#20] + +(31) CometBroadcastHashJoin +Left output [2]: [item_id#21, sr_item_qty#22] +Right output [2]: [item_id#19, cr_item_qty#20] +Arguments: [item_id#21], [item_id#19], Inner, BuildRight + +(32) CometProject +Input [4]: [item_id#21, sr_item_qty#22, item_id#19, cr_item_qty#20] +Arguments: [item_id#21, sr_item_qty#22, cr_item_qty#20], [item_id#21, sr_item_qty#22, cr_item_qty#20] + +(33) Scan parquet spark_catalog.default.web_returns +Output [3]: [wr_item_sk#23, wr_return_quantity#24, wr_returned_date_sk#25] 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#25), dynamicpruningexpression(wr_returned_date_sk#25 IN dynamicpruning#26)] PushedFilters: [IsNotNull(wr_item_sk)] ReadSchema: struct -(36) CometFilter -Input [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] -Condition : isnotnull(wr_item_sk#25) +(34) CometFilter +Input [3]: [wr_item_sk#23, wr_return_quantity#24, wr_returned_date_sk#25] +Condition : isnotnull(wr_item_sk#23) + +(35) ReusedExchange [Reuses operator id: 5] +Output [2]: [i_item_sk#27, i_item_id#28] + +(36) CometBroadcastHashJoin +Left output [3]: [wr_item_sk#23, wr_return_quantity#24, wr_returned_date_sk#25] +Right output [2]: [i_item_sk#27, i_item_id#28] +Arguments: [wr_item_sk#23], [i_item_sk#27], Inner, BuildRight -(37) ReusedExchange [Reuses operator id: 5] -Output [2]: [i_item_sk#29, i_item_id#30] +(37) CometProject +Input [5]: [wr_item_sk#23, wr_return_quantity#24, wr_returned_date_sk#25, i_item_sk#27, i_item_id#28] +Arguments: [wr_return_quantity#24, wr_returned_date_sk#25, i_item_id#28], [wr_return_quantity#24, wr_returned_date_sk#25, i_item_id#28] -(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 +(38) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_date_sk#29] -(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] +(39) CometBroadcastHashJoin +Left output [3]: [wr_return_quantity#24, wr_returned_date_sk#25, i_item_id#28] +Right output [1]: [d_date_sk#29] +Arguments: [wr_returned_date_sk#25], [d_date_sk#29], Inner, BuildRight -(40) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#31] +(40) CometProject +Input [4]: [wr_return_quantity#24, wr_returned_date_sk#25, i_item_id#28, d_date_sk#29] +Arguments: [wr_return_quantity#24, i_item_id#28], [wr_return_quantity#24, i_item_id#28] -(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 +(41) CometHashAggregate +Input [2]: [wr_return_quantity#24, i_item_id#28] +Keys [1]: [i_item_id#28] +Functions [1]: [partial_sum(wr_return_quantity#24)] -(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] +(42) CometColumnarExchange +Input [2]: [i_item_id#28, sum#30] +Arguments: hashpartitioning(i_item_id#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (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)] - -(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] - -(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)#33] -Results [2]: [i_item_id#30 AS item_id#34, sum(wr_return_quantity#26)#33 AS wr_item_qty#35] - -(47) BroadcastExchange -Input [2]: [item_id#34, wr_item_qty#35] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] - -(48) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [item_id#12] -Right keys [1]: [item_id#34] -Join type: Inner -Join condition: None - -(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] - -(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] +Input [2]: [i_item_id#28, sum#30] +Keys [1]: [i_item_id#28] +Functions [1]: [sum(wr_return_quantity#24)] -===== Subqueries ===== +(44) CometBroadcastExchange +Input [2]: [item_id#31, wr_item_qty#32] +Arguments: [item_id#31, wr_item_qty#32] -Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 -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) +(45) CometBroadcastHashJoin +Left output [3]: [item_id#21, sr_item_qty#22, cr_item_qty#20] +Right output [2]: [item_id#31, wr_item_qty#32] +Arguments: [item_id#21], [item_id#31], Inner, BuildRight +(46) CometProject +Input [5]: [item_id#21, sr_item_qty#22, cr_item_qty#20, item_id#31, wr_item_qty#32] +Arguments: [item_id#21, sr_item_qty#22, sr_dev#33, cr_item_qty#20, cr_dev#34, wr_item_qty#32, wr_dev#35, average#36], [item_id#21, sr_item_qty#22, (((cast(sr_item_qty#22 as double) / cast(((sr_item_qty#22 + cr_item_qty#20) + wr_item_qty#32) as double)) / 3.0) * 100.0) AS sr_dev#33, cr_item_qty#20, (((cast(cr_item_qty#20 as double) / cast(((sr_item_qty#22 + cr_item_qty#20) + wr_item_qty#32) as double)) / 3.0) * 100.0) AS cr_dev#34, wr_item_qty#32, (((cast(wr_item_qty#32 as double) / cast(((sr_item_qty#22 + cr_item_qty#20) + wr_item_qty#32) as double)) / 3.0) * 100.0) AS wr_dev#35, (cast(((sr_item_qty#22 + cr_item_qty#20) + wr_item_qty#32) as decimal(20,0)) / 3.0) AS average#36] -(51) Scan parquet spark_catalog.default.date_dim +(47) CometTakeOrderedAndProject +Input [8]: [item_id#21, sr_item_qty#22, sr_dev#33, cr_item_qty#20, cr_dev#34, wr_item_qty#32, wr_dev#35, average#36] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_id#21 ASC NULLS FIRST,sr_item_qty#22 ASC NULLS FIRST], output=[item_id#21,sr_item_qty#22,sr_dev#33,cr_item_qty#20,cr_dev#34,wr_item_qty#32,wr_dev#35,average#36]), [item_id#21, sr_item_qty#22, sr_dev#33, cr_item_qty#20, cr_dev#34, wr_item_qty#32, wr_dev#35, average#36], 100, [item_id#21 ASC NULLS FIRST, sr_item_qty#22 ASC NULLS FIRST], [item_id#21, sr_item_qty#22, sr_dev#33, cr_item_qty#20, cr_dev#34, wr_item_qty#32, wr_dev#35, average#36] + +(48) ColumnarToRow [codegen id : 1] +Input [8]: [item_id#21, sr_item_qty#22, sr_dev#33, cr_item_qty#20, cr_dev#34, wr_item_qty#32, wr_dev#35, average#36] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 +BroadcastExchange (62) ++- * ColumnarToRow (61) + +- CometProject (60) + +- CometBroadcastHashJoin (59) + :- CometFilter (50) + : +- CometScan parquet spark_catalog.default.date_dim (49) + +- CometBroadcastExchange (58) + +- CometProject (57) + +- CometBroadcastHashJoin (56) + :- CometScan parquet spark_catalog.default.date_dim (51) + +- CometBroadcastExchange (55) + +- CometProject (54) + +- CometFilter (53) + +- CometScan parquet spark_catalog.default.date_dim (52) + + +(49) 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 -(52) CometFilter +(50) CometFilter Input [2]: [d_date_sk#7, d_date#8] Condition : isnotnull(d_date_sk#7) -(53) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date#9, d_week_seq#40] +(51) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date#9, d_week_seq#37] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] ReadSchema: struct -(54) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date#41, d_week_seq#42] +(52) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date#38, d_week_seq#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] ReadSchema: struct -(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) +(53) CometFilter +Input [2]: [d_date#38, d_week_seq#39] +Condition : cast(d_date#38 as string) IN (2000-06-30,2000-09-27,2000-11-17) -(56) CometProject -Input [2]: [d_date#41, d_week_seq#42] -Arguments: [d_week_seq#42], [d_week_seq#42] +(54) CometProject +Input [2]: [d_date#38, d_week_seq#39] +Arguments: [d_week_seq#39], [d_week_seq#39] -(57) CometBroadcastExchange -Input [1]: [d_week_seq#42] -Arguments: [d_week_seq#42] +(55) CometBroadcastExchange +Input [1]: [d_week_seq#39] +Arguments: [d_week_seq#39] -(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 +(56) CometBroadcastHashJoin +Left output [2]: [d_date#9, d_week_seq#37] +Right output [1]: [d_week_seq#39] +Arguments: [d_week_seq#37], [d_week_seq#39], LeftSemi, BuildRight -(59) CometProject -Input [2]: [d_date#9, d_week_seq#40] +(57) CometProject +Input [2]: [d_date#9, d_week_seq#37] Arguments: [d_date#9], [d_date#9] -(60) CometBroadcastExchange +(58) CometBroadcastExchange Input [1]: [d_date#9] Arguments: [d_date#9] -(61) CometBroadcastHashJoin +(59) 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 -(62) CometProject +(60) CometProject Input [2]: [d_date_sk#7, d_date#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(63) ColumnarToRow [codegen id : 1] +(61) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(64) BroadcastExchange +(62) 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] -Subquery:2 Hosting operator id = 20 Hosting Expression = cr_returned_date_sk#16 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 19 Hosting Expression = cr_returned_date_sk#13 IN dynamicpruning#4 -Subquery:3 Hosting operator id = 35 Hosting Expression = wr_returned_date_sk#27 IN dynamicpruning#4 +Subquery:3 Hosting operator id = 33 Hosting Expression = wr_returned_date_sk#25 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/simplified.txt index 390c20b7f..8365f11a4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/simplified.txt @@ -1,83 +1,69 @@ -TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty,wr_dev,average] - 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] - BroadcastHashJoin [item_id,item_id] - HashAggregate [i_item_id,sum] [sum(sr_return_quantity),item_id,sr_item_qty,sum] - InputAdapter - Exchange [i_item_id] #1 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,sum,sr_return_quantity] - CometProject [sr_return_quantity,i_item_id] - CometBroadcastHashJoin [sr_return_quantity,sr_returned_date_sk,i_item_id,d_date_sk] - CometProject [sr_return_quantity,sr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [sr_item_sk,sr_return_quantity,sr_returned_date_sk,i_item_sk,i_item_id] - CometFilter [sr_item_sk,sr_return_quantity,sr_returned_date_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 (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] - CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] - CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] - CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [i_item_sk,i_item_id] #5 - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date] #3 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - HashAggregate [i_item_id,sum] [sum(cr_return_quantity),item_id,cr_item_qty,sum] - InputAdapter - Exchange [i_item_id] #8 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,sum,cr_return_quantity] - CometProject [cr_return_quantity,i_item_id] - CometBroadcastHashJoin [cr_return_quantity,cr_returned_date_sk,i_item_id,d_date_sk] - CometProject [cr_return_quantity,cr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [cr_item_sk,cr_return_quantity,cr_returned_date_sk,i_item_sk,i_item_id] - CometFilter [cr_item_sk,cr_return_quantity,cr_returned_date_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [i_item_sk,i_item_id] #5 - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (5) - HashAggregate [i_item_id,sum] [sum(wr_return_quantity),item_id,wr_item_qty,sum] - InputAdapter - Exchange [i_item_id] #10 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,sum,wr_return_quantity] - CometProject [wr_return_quantity,i_item_id] - CometBroadcastHashJoin [wr_return_quantity,wr_returned_date_sk,i_item_id,d_date_sk] - CometProject [wr_return_quantity,wr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [wr_item_sk,wr_return_quantity,wr_returned_date_sk,i_item_sk,i_item_id] - CometFilter [wr_item_sk,wr_return_quantity,wr_returned_date_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [i_item_sk,i_item_id] #5 - ReusedExchange [d_date_sk] #6 +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty,wr_dev,average] + CometProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty,wr_dev,average] + CometBroadcastHashJoin [item_id,sr_item_qty,cr_item_qty,item_id,wr_item_qty] + CometProject [item_id,sr_item_qty,cr_item_qty] + CometBroadcastHashJoin [item_id,sr_item_qty,item_id,cr_item_qty] + CometHashAggregate [item_id,sr_item_qty,i_item_id,sum,sum(sr_return_quantity)] + CometColumnarExchange [i_item_id] #1 + CometHashAggregate [i_item_id,sum,sr_return_quantity] + CometProject [sr_return_quantity,i_item_id] + CometBroadcastHashJoin [sr_return_quantity,sr_returned_date_sk,i_item_id,d_date_sk] + CometProject [sr_return_quantity,sr_returned_date_sk,i_item_id] + CometBroadcastHashJoin [sr_item_sk,sr_return_quantity,sr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [sr_item_sk,sr_return_quantity,sr_returned_date_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 (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date] #3 + CometProject [d_date] + CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometBroadcastExchange [d_week_seq] #4 + CometProject [d_week_seq] + CometFilter [d_date,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometBroadcastExchange [i_item_sk,i_item_id] #5 + CometFilter [i_item_sk,i_item_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + ReusedExchange [d_date] #3 + CometBroadcastExchange [item_id,cr_item_qty] #7 + CometHashAggregate [item_id,cr_item_qty,i_item_id,sum,sum(cr_return_quantity)] + CometColumnarExchange [i_item_id] #8 + CometHashAggregate [i_item_id,sum,cr_return_quantity] + CometProject [cr_return_quantity,i_item_id] + CometBroadcastHashJoin [cr_return_quantity,cr_returned_date_sk,i_item_id,d_date_sk] + CometProject [cr_return_quantity,cr_returned_date_sk,i_item_id] + CometBroadcastHashJoin [cr_item_sk,cr_return_quantity,cr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [cr_item_sk,cr_return_quantity,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [i_item_sk,i_item_id] #5 + ReusedExchange [d_date_sk] #6 + CometBroadcastExchange [item_id,wr_item_qty] #9 + CometHashAggregate [item_id,wr_item_qty,i_item_id,sum,sum(wr_return_quantity)] + CometColumnarExchange [i_item_id] #10 + CometHashAggregate [i_item_id,sum,wr_return_quantity] + CometProject [wr_return_quantity,i_item_id] + CometBroadcastHashJoin [wr_return_quantity,wr_returned_date_sk,i_item_id,d_date_sk] + CometProject [wr_return_quantity,wr_returned_date_sk,i_item_id] + CometBroadcastHashJoin [wr_item_sk,wr_return_quantity,wr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [wr_item_sk,wr_return_quantity,wr_returned_date_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + 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-spark3_5/q85/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/explain.txt index cdd66620e..968c7402e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (45) -+- * HashAggregate (44) - +- Exchange (43) - +- * ColumnarToRow (42) +* ColumnarToRow (45) ++- CometTakeOrderedAndProject (44) + +- CometHashAggregate (43) + +- CometColumnarExchange (42) +- CometHashAggregate (41) +- CometProject (40) +- CometBroadcastHashJoin (39) @@ -243,23 +243,21 @@ 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))] -(42) ColumnarToRow [codegen id : 1] +(42) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(44) HashAggregate [codegen id : 2] +(43) CometHashAggregate 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)#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] -(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] +(44) CometTakeOrderedAndProject +Input [4]: [substr(r_reason_desc, 1, 20)#38, avg(ws_quantity)#39, avg(wr_refunded_cash)#40, avg(wr_fee)#41] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(r_reason_desc, 1, 20)#38 ASC NULLS FIRST,avg(ws_quantity)#39 ASC NULLS FIRST,avg(wr_refunded_cash)#40 ASC NULLS FIRST,avg(wr_fee)#41 ASC NULLS FIRST], output=[substr(r_reason_desc, 1, 20)#38,avg(ws_quantity)#39,avg(wr_refunded_cash)#40,avg(wr_fee)#41]), [substr(r_reason_desc, 1, 20)#38, avg(ws_quantity)#39, avg(wr_refunded_cash)#40, avg(wr_fee)#41], 100, [substr(r_reason_desc, 1, 20)#38 ASC NULLS FIRST, avg(ws_quantity)#39 ASC NULLS FIRST, avg(wr_refunded_cash)#40 ASC NULLS FIRST, avg(wr_fee)#41 ASC NULLS FIRST], [substr(r_reason_desc, 1, 20)#38, avg(ws_quantity)#39, avg(wr_refunded_cash)#40, avg(wr_fee)#41] + +(45) ColumnarToRow [codegen id : 1] +Input [4]: [substr(r_reason_desc, 1, 20)#38, avg(ws_quantity)#39, avg(wr_refunded_cash)#40, avg(wr_fee)#41] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/simplified.txt index 2313467ae..9467b5c77 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/simplified.txt @@ -1,57 +1,55 @@ -TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee)] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [r_reason_desc,sum,count,sum,count,sum,count,ws_quantity,wr_refunded_cash,wr_fee] - CometProject [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] - CometBroadcastHashJoin [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash,r_reason_sk,r_reason_desc] - CometProject [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash,d_date_sk] - CometProject [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash,ca_address_sk,ca_state] - CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [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,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 [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,cd_demo_sk,cd_marital_status,cd_education_status] - 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,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,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_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,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] - CometBroadcastExchange [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] #2 - CometFilter [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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_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] - 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] - CometBroadcastExchange [wp_web_page_sk] #4 - CometFilter [wp_web_page_sk] - CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #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] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee)] + CometHashAggregate [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee),r_reason_desc,sum,count,sum,count,sum,count,avg(ws_quantity),avg(UnscaledValue(wr_refunded_cash)),avg(UnscaledValue(wr_fee))] + CometColumnarExchange [r_reason_desc] #1 + CometHashAggregate [r_reason_desc,sum,count,sum,count,sum,count,ws_quantity,wr_refunded_cash,wr_fee] + CometProject [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] + CometBroadcastHashJoin [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash,r_reason_sk,r_reason_desc] + CometProject [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash] + CometBroadcastHashJoin [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash,d_date_sk] + CometProject [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash] + CometBroadcastHashJoin [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash,ca_address_sk,ca_state] + CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash] + CometBroadcastHashJoin [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,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 [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,cd_demo_sk,cd_marital_status,cd_education_status] + 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,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,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_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,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] + CometBroadcastExchange [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] #2 + CometFilter [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + 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_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] + 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] + CometBroadcastExchange [wp_web_page_sk] #4 + CometFilter [wp_web_page_sk] + CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #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] - CometBroadcastExchange [ca_address_sk,ca_state] #7 - CometProject [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #8 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [r_reason_sk,r_reason_desc] #9 - CometFilter [r_reason_sk,r_reason_desc] - CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #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 [ca_address_sk,ca_state] #7 + CometProject [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + CometBroadcastExchange [d_date_sk] #8 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [r_reason_sk,r_reason_desc] #9 + CometFilter [r_reason_sk,r_reason_desc] + 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-spark3_5/q86/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/explain.txt index 900d23064..3d0fe32c7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/explain.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject (23) +- * Project (22) +- Window (21) - +- * Sort (20) - +- Exchange (19) - +- * HashAggregate (18) - +- Exchange (17) - +- * ColumnarToRow (16) + +- * ColumnarToRow (20) + +- CometSort (19) + +- CometColumnarExchange (18) + +- CometHashAggregate (17) + +- CometColumnarExchange (16) +- CometHashAggregate (15) +- CometExpand (14) +- CometProject (13) @@ -97,39 +97,37 @@ 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))] -(16) ColumnarToRow [codegen id : 1] +(16) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(17) CometHashAggregate 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#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] -(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) CometColumnarExchange +Input [7]: [total_sum#14, i_category#10, i_class#11, lochierarchy#15, _w0#16, _w1#17, _w2#18] +Arguments: hashpartitioning(_w1#17, _w2#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(19) CometSort +Input [7]: [total_sum#14, i_category#10, i_class#11, lochierarchy#15, _w0#16, _w1#17, _w2#18] +Arguments: [total_sum#14, i_category#10, i_class#11, lochierarchy#15, _w0#16, _w1#17, _w2#18], [_w1#17 ASC NULLS FIRST, _w2#18 ASC NULLS FIRST, _w0#16 DESC NULLS LAST] -(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 +(20) ColumnarToRow [codegen id : 1] +Input [7]: [total_sum#14, i_category#10, i_class#11, lochierarchy#15, _w0#16, _w1#17, _w2#18] (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] +Input [7]: [total_sum#14, i_category#10, i_class#11, lochierarchy#15, _w0#16, _w1#17, _w2#18] +Arguments: [rank(_w0#16) windowspecdefinition(_w1#17, _w2#18, _w0#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#19], [_w1#17, _w2#18], [_w0#16 DESC NULLS LAST] -(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] +(22) Project [codegen id : 2] +Output [5]: [total_sum#14, i_category#10, i_class#11, lochierarchy#15, rank_within_parent#19] +Input [8]: [total_sum#14, i_category#10, i_class#11, lochierarchy#15, _w0#16, _w1#17, _w2#18, rank_within_parent#19] (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] +Input [5]: [total_sum#14, i_category#10, i_class#11, lochierarchy#15, rank_within_parent#19] +Arguments: 100, [lochierarchy#15 DESC NULLS LAST, CASE WHEN (lochierarchy#15 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#19 ASC NULLS FIRST], [total_sum#14, i_category#10, i_class#11, lochierarchy#15, rank_within_parent#19] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/simplified.txt index a6d1c25fc..bbe4a2d2f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/simplified.txt @@ -1,39 +1,35 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (4) + WholeStageCodegen (2) Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [_w0,_w1,_w2] - WholeStageCodegen (3) - Sort [_w1,_w2,_w0] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [_w1,_w2] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_category,i_class,spark_grouping_id,sum,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,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometFilter [i_item_sk,i_class,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] + CometColumnarExchange [_w1,_w2] #1 + CometHashAggregate [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2,spark_grouping_id,sum,sum(UnscaledValue(ws_net_paid))] + CometColumnarExchange [i_category,i_class,spark_grouping_id] #2 + CometHashAggregate [i_category,i_class,spark_grouping_id,sum,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,ws_net_paid,i_item_sk,i_class,i_category] + CometProject [ws_item_sk,ws_net_paid] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometFilter [i_item_sk,i_class,i_category] + 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-spark3_5/q87/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/explain.txt index 0434066e6..d7ea7098e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/explain.txt @@ -1,53 +1,55 @@ == Physical Plan == -* 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) +* HashAggregate (51) ++- * ColumnarToRow (50) + +- CometColumnarExchange (49) + +- RowToColumnar (48) + +- * HashAggregate (47) + +- * Project (46) + +- * BroadcastHashJoin LeftAnti BuildRight (45) + :- * BroadcastHashJoin LeftAnti BuildRight (31) + : :- * ColumnarToRow (17) + : : +- CometHashAggregate (16) + : : +- CometColumnarExchange (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) + : +- * ColumnarToRow (29) + : +- CometHashAggregate (28) + : +- CometColumnarExchange (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) + +- * ColumnarToRow (43) + +- CometHashAggregate (42) + +- CometColumnarExchange (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 @@ -119,19 +121,17 @@ 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: [] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(17) HashAggregate [codegen id : 6] +(16) 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#9, c_first_name#8, d_date#5] + +(17) ColumnarToRow [codegen id : 3] +Input [3]: [c_last_name#9, c_first_name#8, d_date#5] (18) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_bill_customer_sk#10, cs_sold_date_sk#11] @@ -174,25 +174,23 @@ 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: [] -(27) ColumnarToRow [codegen id : 2] +(27) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=2] -(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] +(28) 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#17, c_first_name#16, d_date#14] + +(29) ColumnarToRow [codegen id : 1] +Input [3]: [c_last_name#17, c_first_name#16, d_date#14] (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] -(31) BroadcastHashJoin [codegen id : 6] +(31) BroadcastHashJoin [codegen id : 3] 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 @@ -239,46 +237,50 @@ 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: [] -(41) ColumnarToRow [codegen id : 4] +(41) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=4] -(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] - -(43) HashAggregate [codegen id : 5] +(42) 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#25, c_first_name#24, d_date#22] + +(43) ColumnarToRow [codegen id : 2] +Input [3]: [c_last_name#25, c_first_name#24, d_date#22] (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] -(45) BroadcastHashJoin [codegen id : 6] +(45) BroadcastHashJoin [codegen id : 3] 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 -(46) Project [codegen id : 6] +(46) Project [codegen id : 3] Output: [] Input [3]: [c_last_name#9, c_first_name#8, d_date#5] -(47) HashAggregate [codegen id : 6] +(47) HashAggregate [codegen id : 3] Input: [] Keys: [] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#26] Results [1]: [count#27] -(48) Exchange +(48) RowToColumnar +Input [1]: [count#27] + +(49) CometColumnarExchange +Input [1]: [count#27] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(50) ColumnarToRow [codegen id : 4] Input [1]: [count#27] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(49) HashAggregate [codegen id : 7] +(51) HashAggregate [codegen id : 4] Input [1]: [count#27] Keys: [] Functions [1]: [count(1)] @@ -288,32 +290,32 @@ 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 (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 +(52) 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 -(51) CometFilter +(53) 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)) -(52) CometProject +(54) 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] -(53) ColumnarToRow [codegen id : 1] +(55) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#4, d_date#5] -(54) BroadcastExchange +(56) BroadcastExchange Input [2]: [d_date_sk#4, d_date#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/simplified.txt index 9d667265c..69f33a57a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/simplified.txt @@ -1,18 +1,18 @@ -WholeStageCodegen (7) +WholeStageCodegen (4) HashAggregate [count] [count(1),count(1),count] - InputAdapter - Exchange #1 - WholeStageCodegen (6) - HashAggregate [count,count] - Project - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometColumnarExchange #1 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [count,count] + Project + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + ColumnarToRow + InputAdapter + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #2 CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] @@ -35,15 +35,13 @@ WholeStageCodegen (7) CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #7 CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] CometBroadcastHashJoin [cs_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] @@ -54,15 +52,13 @@ WholeStageCodegen (7) ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #9 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #9 CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] CometBroadcastHashJoin [ws_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/explain.txt index 3f905ebe9..cd9e4817c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/explain.txt @@ -6,9 +6,9 @@ : : : :- * BroadcastNestedLoopJoin Inner BuildRight (88) : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (67) : : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (46) -: : : : : : :- * HashAggregate (25) -: : : : : : : +- Exchange (24) -: : : : : : : +- * ColumnarToRow (23) +: : : : : : :- * ColumnarToRow (25) +: : : : : : : +- CometHashAggregate (24) +: : : : : : : +- CometColumnarExchange (23) : : : : : : : +- CometHashAggregate (22) : : : : : : : +- CometProject (21) : : : : : : : +- CometBroadcastHashJoin (20) @@ -32,9 +32,9 @@ : : : : : : : +- CometFilter (17) : : : : : : : +- CometScan parquet spark_catalog.default.store (16) : : : : : : +- BroadcastExchange (45) -: : : : : : +- * HashAggregate (44) -: : : : : : +- Exchange (43) -: : : : : : +- * ColumnarToRow (42) +: : : : : : +- * ColumnarToRow (44) +: : : : : : +- CometHashAggregate (43) +: : : : : : +- CometColumnarExchange (42) : : : : : : +- CometHashAggregate (41) : : : : : : +- CometProject (40) : : : : : : +- CometBroadcastHashJoin (39) @@ -52,9 +52,9 @@ : : : : : : : +- CometScan parquet spark_catalog.default.time_dim (32) : : : : : : +- ReusedExchange (38) : : : : : +- BroadcastExchange (66) -: : : : : +- * HashAggregate (65) -: : : : : +- Exchange (64) -: : : : : +- * ColumnarToRow (63) +: : : : : +- * ColumnarToRow (65) +: : : : : +- CometHashAggregate (64) +: : : : : +- CometColumnarExchange (63) : : : : : +- CometHashAggregate (62) : : : : : +- CometProject (61) : : : : : +- CometBroadcastHashJoin (60) @@ -72,9 +72,9 @@ : : : : : : +- CometScan parquet spark_catalog.default.time_dim (53) : : : : : +- ReusedExchange (59) : : : : +- BroadcastExchange (87) -: : : : +- * HashAggregate (86) -: : : : +- Exchange (85) -: : : : +- * ColumnarToRow (84) +: : : : +- * ColumnarToRow (86) +: : : : +- CometHashAggregate (85) +: : : : +- CometColumnarExchange (84) : : : : +- CometHashAggregate (83) : : : : +- CometProject (82) : : : : +- CometBroadcastHashJoin (81) @@ -92,9 +92,9 @@ : : : : : +- CometScan parquet spark_catalog.default.time_dim (74) : : : : +- ReusedExchange (80) : : : +- BroadcastExchange (108) -: : : +- * HashAggregate (107) -: : : +- Exchange (106) -: : : +- * ColumnarToRow (105) +: : : +- * ColumnarToRow (107) +: : : +- CometHashAggregate (106) +: : : +- CometColumnarExchange (105) : : : +- CometHashAggregate (104) : : : +- CometProject (103) : : : +- CometBroadcastHashJoin (102) @@ -112,9 +112,9 @@ : : : : +- CometScan parquet spark_catalog.default.time_dim (95) : : : +- ReusedExchange (101) : : +- BroadcastExchange (129) -: : +- * HashAggregate (128) -: : +- Exchange (127) -: : +- * ColumnarToRow (126) +: : +- * ColumnarToRow (128) +: : +- CometHashAggregate (127) +: : +- CometColumnarExchange (126) : : +- CometHashAggregate (125) : : +- CometProject (124) : : +- CometBroadcastHashJoin (123) @@ -132,9 +132,9 @@ : : : +- CometScan parquet spark_catalog.default.time_dim (116) : : +- ReusedExchange (122) : +- BroadcastExchange (150) -: +- * HashAggregate (149) -: +- Exchange (148) -: +- * ColumnarToRow (147) +: +- * ColumnarToRow (149) +: +- CometHashAggregate (148) +: +- CometColumnarExchange (147) : +- CometHashAggregate (146) : +- CometProject (145) : +- CometBroadcastHashJoin (144) @@ -152,9 +152,9 @@ : : +- CometScan parquet spark_catalog.default.time_dim (137) : +- ReusedExchange (143) +- BroadcastExchange (171) - +- * HashAggregate (170) - +- Exchange (169) - +- * ColumnarToRow (168) + +- * ColumnarToRow (170) + +- CometHashAggregate (169) + +- CometColumnarExchange (168) +- CometHashAggregate (167) +- CometProject (166) +- CometBroadcastHashJoin (165) @@ -276,668 +276,652 @@ Input: [] Keys: [] Functions [1]: [partial_count(1)] -(23) ColumnarToRow [codegen id : 1] +(23) CometColumnarExchange Input [1]: [count#13] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(24) Exchange -Input [1]: [count#13] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] - -(25) HashAggregate [codegen id : 16] +(24) CometHashAggregate Input [1]: [count#13] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#14] -Results [1]: [count(1)#14 AS h8_30_to_9#15] + +(25) ColumnarToRow [codegen id : 8] +Input [1]: [h8_30_to_9#14] (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] +Output [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] 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 (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)) +Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] +Condition : ((isnotnull(ss_hdemo_sk#16) AND isnotnull(ss_sold_time_sk#15)) AND isnotnull(ss_store_sk#17)) (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] +Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] +Arguments: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17], [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17] (29) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#20] +Output [1]: [hd_demo_sk#19] (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 +Left output [3]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17] +Right output [1]: [hd_demo_sk#19] +Arguments: [ss_hdemo_sk#16], [hd_demo_sk#19], Inner, BuildRight (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] +Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, hd_demo_sk#19] +Arguments: [ss_sold_time_sk#15, ss_store_sk#17], [ss_sold_time_sk#15, ss_store_sk#17] (32) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#21, t_hour#22, t_minute#23] +Output [3]: [t_time_sk#20, t_hour#21, t_minute#22] 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 (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)) +Input [3]: [t_time_sk#20, t_hour#21, t_minute#22] +Condition : ((((isnotnull(t_hour#21) AND isnotnull(t_minute#22)) AND (t_hour#21 = 9)) AND (t_minute#22 < 30)) AND isnotnull(t_time_sk#20)) (34) CometProject -Input [3]: [t_time_sk#21, t_hour#22, t_minute#23] -Arguments: [t_time_sk#21], [t_time_sk#21] +Input [3]: [t_time_sk#20, t_hour#21, t_minute#22] +Arguments: [t_time_sk#20], [t_time_sk#20] (35) CometBroadcastExchange -Input [1]: [t_time_sk#21] -Arguments: [t_time_sk#21] +Input [1]: [t_time_sk#20] +Arguments: [t_time_sk#20] (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 +Left output [2]: [ss_sold_time_sk#15, ss_store_sk#17] +Right output [1]: [t_time_sk#20] +Arguments: [ss_sold_time_sk#15], [t_time_sk#20], Inner, BuildRight (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] +Input [3]: [ss_sold_time_sk#15, ss_store_sk#17, t_time_sk#20] +Arguments: [ss_store_sk#17], [ss_store_sk#17] (38) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#24] +Output [1]: [s_store_sk#23] (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 +Left output [1]: [ss_store_sk#17] +Right output [1]: [s_store_sk#23] +Arguments: [ss_store_sk#17], [s_store_sk#23], Inner, BuildRight (40) CometProject -Input [2]: [ss_store_sk#18, s_store_sk#24] +Input [2]: [ss_store_sk#17, s_store_sk#23] (41) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -(42) ColumnarToRow [codegen id : 2] -Input [1]: [count#25] +(42) CometColumnarExchange +Input [1]: [count#24] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(43) Exchange -Input [1]: [count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] - -(44) HashAggregate [codegen id : 3] -Input [1]: [count#25] +(43) CometHashAggregate +Input [1]: [count#24] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#26] -Results [1]: [count(1)#26 AS h9_to_9_30#27] + +(44) ColumnarToRow [codegen id : 1] +Input [1]: [h9_to_9_30#25] (45) BroadcastExchange -Input [1]: [h9_to_9_30#27] +Input [1]: [h9_to_9_30#25] Arguments: IdentityBroadcastMode, [plan_id=3] -(46) BroadcastNestedLoopJoin [codegen id : 16] +(46) BroadcastNestedLoopJoin [codegen id : 8] Join type: Inner Join condition: None (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] +Output [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] 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 (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)) +Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] +Condition : ((isnotnull(ss_hdemo_sk#27) AND isnotnull(ss_sold_time_sk#26)) AND isnotnull(ss_store_sk#28)) (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] +Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] +Arguments: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28], [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28] (50) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#32] +Output [1]: [hd_demo_sk#30] (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 +Left output [3]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28] +Right output [1]: [hd_demo_sk#30] +Arguments: [ss_hdemo_sk#27], [hd_demo_sk#30], Inner, BuildRight (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] +Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, hd_demo_sk#30] +Arguments: [ss_sold_time_sk#26, ss_store_sk#28], [ss_sold_time_sk#26, ss_store_sk#28] (53) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#33, t_hour#34, t_minute#35] +Output [3]: [t_time_sk#31, t_hour#32, t_minute#33] 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 (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)) +Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] +Condition : ((((isnotnull(t_hour#32) AND isnotnull(t_minute#33)) AND (t_hour#32 = 9)) AND (t_minute#33 >= 30)) AND isnotnull(t_time_sk#31)) (55) CometProject -Input [3]: [t_time_sk#33, t_hour#34, t_minute#35] -Arguments: [t_time_sk#33], [t_time_sk#33] +Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] +Arguments: [t_time_sk#31], [t_time_sk#31] (56) CometBroadcastExchange -Input [1]: [t_time_sk#33] -Arguments: [t_time_sk#33] +Input [1]: [t_time_sk#31] +Arguments: [t_time_sk#31] (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 +Left output [2]: [ss_sold_time_sk#26, ss_store_sk#28] +Right output [1]: [t_time_sk#31] +Arguments: [ss_sold_time_sk#26], [t_time_sk#31], Inner, BuildRight (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] +Input [3]: [ss_sold_time_sk#26, ss_store_sk#28, t_time_sk#31] +Arguments: [ss_store_sk#28], [ss_store_sk#28] (59) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#36] +Output [1]: [s_store_sk#34] (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 +Left output [1]: [ss_store_sk#28] +Right output [1]: [s_store_sk#34] +Arguments: [ss_store_sk#28], [s_store_sk#34], Inner, BuildRight (61) CometProject -Input [2]: [ss_store_sk#30, s_store_sk#36] +Input [2]: [ss_store_sk#28, s_store_sk#34] (62) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -(63) ColumnarToRow [codegen id : 4] -Input [1]: [count#37] +(63) CometColumnarExchange +Input [1]: [count#35] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(64) Exchange -Input [1]: [count#37] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] - -(65) HashAggregate [codegen id : 5] -Input [1]: [count#37] +(64) CometHashAggregate +Input [1]: [count#35] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#38] -Results [1]: [count(1)#38 AS h9_30_to_10#39] + +(65) ColumnarToRow [codegen id : 2] +Input [1]: [h9_30_to_10#36] (66) BroadcastExchange -Input [1]: [h9_30_to_10#39] +Input [1]: [h9_30_to_10#36] Arguments: IdentityBroadcastMode, [plan_id=5] -(67) BroadcastNestedLoopJoin [codegen id : 16] +(67) BroadcastNestedLoopJoin [codegen id : 8] Join type: Inner Join condition: None (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] +Output [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] 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 (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)) +Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] +Condition : ((isnotnull(ss_hdemo_sk#38) AND isnotnull(ss_sold_time_sk#37)) AND isnotnull(ss_store_sk#39)) (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] +Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] +Arguments: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39], [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39] (71) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#44] +Output [1]: [hd_demo_sk#41] (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 +Left output [3]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39] +Right output [1]: [hd_demo_sk#41] +Arguments: [ss_hdemo_sk#38], [hd_demo_sk#41], Inner, BuildRight (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] +Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, hd_demo_sk#41] +Arguments: [ss_sold_time_sk#37, ss_store_sk#39], [ss_sold_time_sk#37, ss_store_sk#39] (74) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#45, t_hour#46, t_minute#47] +Output [3]: [t_time_sk#42, t_hour#43, t_minute#44] 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 (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)) +Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] +Condition : ((((isnotnull(t_hour#43) AND isnotnull(t_minute#44)) AND (t_hour#43 = 10)) AND (t_minute#44 < 30)) AND isnotnull(t_time_sk#42)) (76) CometProject -Input [3]: [t_time_sk#45, t_hour#46, t_minute#47] -Arguments: [t_time_sk#45], [t_time_sk#45] +Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] +Arguments: [t_time_sk#42], [t_time_sk#42] (77) CometBroadcastExchange -Input [1]: [t_time_sk#45] -Arguments: [t_time_sk#45] +Input [1]: [t_time_sk#42] +Arguments: [t_time_sk#42] (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 +Left output [2]: [ss_sold_time_sk#37, ss_store_sk#39] +Right output [1]: [t_time_sk#42] +Arguments: [ss_sold_time_sk#37], [t_time_sk#42], Inner, BuildRight (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] +Input [3]: [ss_sold_time_sk#37, ss_store_sk#39, t_time_sk#42] +Arguments: [ss_store_sk#39], [ss_store_sk#39] (80) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#48] +Output [1]: [s_store_sk#45] (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 +Left output [1]: [ss_store_sk#39] +Right output [1]: [s_store_sk#45] +Arguments: [ss_store_sk#39], [s_store_sk#45], Inner, BuildRight (82) CometProject -Input [2]: [ss_store_sk#42, s_store_sk#48] +Input [2]: [ss_store_sk#39, s_store_sk#45] (83) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -(84) ColumnarToRow [codegen id : 6] -Input [1]: [count#49] +(84) CometColumnarExchange +Input [1]: [count#46] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(85) Exchange -Input [1]: [count#49] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] - -(86) HashAggregate [codegen id : 7] -Input [1]: [count#49] +(85) CometHashAggregate +Input [1]: [count#46] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#50] -Results [1]: [count(1)#50 AS h10_to_10_30#51] + +(86) ColumnarToRow [codegen id : 3] +Input [1]: [h10_to_10_30#47] (87) BroadcastExchange -Input [1]: [h10_to_10_30#51] +Input [1]: [h10_to_10_30#47] Arguments: IdentityBroadcastMode, [plan_id=7] -(88) BroadcastNestedLoopJoin [codegen id : 16] +(88) BroadcastNestedLoopJoin [codegen id : 8] Join type: Inner Join condition: None (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] +Output [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] 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 (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)) +Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] +Condition : ((isnotnull(ss_hdemo_sk#49) AND isnotnull(ss_sold_time_sk#48)) AND isnotnull(ss_store_sk#50)) (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] +Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] +Arguments: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50], [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50] (92) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#56] +Output [1]: [hd_demo_sk#52] (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 +Left output [3]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50] +Right output [1]: [hd_demo_sk#52] +Arguments: [ss_hdemo_sk#49], [hd_demo_sk#52], Inner, BuildRight (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] +Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, hd_demo_sk#52] +Arguments: [ss_sold_time_sk#48, ss_store_sk#50], [ss_sold_time_sk#48, ss_store_sk#50] (95) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#57, t_hour#58, t_minute#59] +Output [3]: [t_time_sk#53, t_hour#54, t_minute#55] 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 (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)) +Input [3]: [t_time_sk#53, t_hour#54, t_minute#55] +Condition : ((((isnotnull(t_hour#54) AND isnotnull(t_minute#55)) AND (t_hour#54 = 10)) AND (t_minute#55 >= 30)) AND isnotnull(t_time_sk#53)) (97) CometProject -Input [3]: [t_time_sk#57, t_hour#58, t_minute#59] -Arguments: [t_time_sk#57], [t_time_sk#57] +Input [3]: [t_time_sk#53, t_hour#54, t_minute#55] +Arguments: [t_time_sk#53], [t_time_sk#53] (98) CometBroadcastExchange -Input [1]: [t_time_sk#57] -Arguments: [t_time_sk#57] +Input [1]: [t_time_sk#53] +Arguments: [t_time_sk#53] (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 +Left output [2]: [ss_sold_time_sk#48, ss_store_sk#50] +Right output [1]: [t_time_sk#53] +Arguments: [ss_sold_time_sk#48], [t_time_sk#53], Inner, BuildRight (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] +Input [3]: [ss_sold_time_sk#48, ss_store_sk#50, t_time_sk#53] +Arguments: [ss_store_sk#50], [ss_store_sk#50] (101) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#60] +Output [1]: [s_store_sk#56] (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 +Left output [1]: [ss_store_sk#50] +Right output [1]: [s_store_sk#56] +Arguments: [ss_store_sk#50], [s_store_sk#56], Inner, BuildRight (103) CometProject -Input [2]: [ss_store_sk#54, s_store_sk#60] +Input [2]: [ss_store_sk#50, s_store_sk#56] (104) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -(105) ColumnarToRow [codegen id : 8] -Input [1]: [count#61] +(105) CometColumnarExchange +Input [1]: [count#57] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(106) Exchange -Input [1]: [count#61] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=8] - -(107) HashAggregate [codegen id : 9] -Input [1]: [count#61] +(106) CometHashAggregate +Input [1]: [count#57] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#62] -Results [1]: [count(1)#62 AS h10_30_to_11#63] + +(107) ColumnarToRow [codegen id : 4] +Input [1]: [h10_30_to_11#58] (108) BroadcastExchange -Input [1]: [h10_30_to_11#63] +Input [1]: [h10_30_to_11#58] Arguments: IdentityBroadcastMode, [plan_id=9] -(109) BroadcastNestedLoopJoin [codegen id : 16] +(109) BroadcastNestedLoopJoin [codegen id : 8] Join type: Inner Join condition: None (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] +Output [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] 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 (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)) +Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] +Condition : ((isnotnull(ss_hdemo_sk#60) AND isnotnull(ss_sold_time_sk#59)) AND isnotnull(ss_store_sk#61)) (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] +Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] +Arguments: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61], [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61] (113) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#68] +Output [1]: [hd_demo_sk#63] (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 +Left output [3]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61] +Right output [1]: [hd_demo_sk#63] +Arguments: [ss_hdemo_sk#60], [hd_demo_sk#63], Inner, BuildRight (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] +Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, hd_demo_sk#63] +Arguments: [ss_sold_time_sk#59, ss_store_sk#61], [ss_sold_time_sk#59, ss_store_sk#61] (116) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#69, t_hour#70, t_minute#71] +Output [3]: [t_time_sk#64, t_hour#65, t_minute#66] 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 (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)) +Input [3]: [t_time_sk#64, t_hour#65, t_minute#66] +Condition : ((((isnotnull(t_hour#65) AND isnotnull(t_minute#66)) AND (t_hour#65 = 11)) AND (t_minute#66 < 30)) AND isnotnull(t_time_sk#64)) (118) CometProject -Input [3]: [t_time_sk#69, t_hour#70, t_minute#71] -Arguments: [t_time_sk#69], [t_time_sk#69] +Input [3]: [t_time_sk#64, t_hour#65, t_minute#66] +Arguments: [t_time_sk#64], [t_time_sk#64] (119) CometBroadcastExchange -Input [1]: [t_time_sk#69] -Arguments: [t_time_sk#69] +Input [1]: [t_time_sk#64] +Arguments: [t_time_sk#64] (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 +Left output [2]: [ss_sold_time_sk#59, ss_store_sk#61] +Right output [1]: [t_time_sk#64] +Arguments: [ss_sold_time_sk#59], [t_time_sk#64], Inner, BuildRight (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] +Input [3]: [ss_sold_time_sk#59, ss_store_sk#61, t_time_sk#64] +Arguments: [ss_store_sk#61], [ss_store_sk#61] (122) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#72] +Output [1]: [s_store_sk#67] (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 +Left output [1]: [ss_store_sk#61] +Right output [1]: [s_store_sk#67] +Arguments: [ss_store_sk#61], [s_store_sk#67], Inner, BuildRight (124) CometProject -Input [2]: [ss_store_sk#66, s_store_sk#72] +Input [2]: [ss_store_sk#61, s_store_sk#67] (125) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -(126) ColumnarToRow [codegen id : 10] -Input [1]: [count#73] +(126) CometColumnarExchange +Input [1]: [count#68] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(127) Exchange -Input [1]: [count#73] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] - -(128) HashAggregate [codegen id : 11] -Input [1]: [count#73] +(127) CometHashAggregate +Input [1]: [count#68] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#74] -Results [1]: [count(1)#74 AS h11_to_11_30#75] + +(128) ColumnarToRow [codegen id : 5] +Input [1]: [h11_to_11_30#69] (129) BroadcastExchange -Input [1]: [h11_to_11_30#75] +Input [1]: [h11_to_11_30#69] Arguments: IdentityBroadcastMode, [plan_id=11] -(130) BroadcastNestedLoopJoin [codegen id : 16] +(130) BroadcastNestedLoopJoin [codegen id : 8] Join type: Inner Join condition: None (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] +Output [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] 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 (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)) +Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] +Condition : ((isnotnull(ss_hdemo_sk#71) AND isnotnull(ss_sold_time_sk#70)) AND isnotnull(ss_store_sk#72)) (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] +Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] +Arguments: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72], [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72] (134) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#80] +Output [1]: [hd_demo_sk#74] (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 +Left output [3]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72] +Right output [1]: [hd_demo_sk#74] +Arguments: [ss_hdemo_sk#71], [hd_demo_sk#74], Inner, BuildRight (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] +Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, hd_demo_sk#74] +Arguments: [ss_sold_time_sk#70, ss_store_sk#72], [ss_sold_time_sk#70, ss_store_sk#72] (137) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#81, t_hour#82, t_minute#83] +Output [3]: [t_time_sk#75, t_hour#76, t_minute#77] 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 (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)) +Input [3]: [t_time_sk#75, t_hour#76, t_minute#77] +Condition : ((((isnotnull(t_hour#76) AND isnotnull(t_minute#77)) AND (t_hour#76 = 11)) AND (t_minute#77 >= 30)) AND isnotnull(t_time_sk#75)) (139) CometProject -Input [3]: [t_time_sk#81, t_hour#82, t_minute#83] -Arguments: [t_time_sk#81], [t_time_sk#81] +Input [3]: [t_time_sk#75, t_hour#76, t_minute#77] +Arguments: [t_time_sk#75], [t_time_sk#75] (140) CometBroadcastExchange -Input [1]: [t_time_sk#81] -Arguments: [t_time_sk#81] +Input [1]: [t_time_sk#75] +Arguments: [t_time_sk#75] (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 +Left output [2]: [ss_sold_time_sk#70, ss_store_sk#72] +Right output [1]: [t_time_sk#75] +Arguments: [ss_sold_time_sk#70], [t_time_sk#75], Inner, BuildRight (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] +Input [3]: [ss_sold_time_sk#70, ss_store_sk#72, t_time_sk#75] +Arguments: [ss_store_sk#72], [ss_store_sk#72] (143) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#84] +Output [1]: [s_store_sk#78] (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 +Left output [1]: [ss_store_sk#72] +Right output [1]: [s_store_sk#78] +Arguments: [ss_store_sk#72], [s_store_sk#78], Inner, BuildRight (145) CometProject -Input [2]: [ss_store_sk#78, s_store_sk#84] +Input [2]: [ss_store_sk#72, s_store_sk#78] (146) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -(147) ColumnarToRow [codegen id : 12] -Input [1]: [count#85] +(147) CometColumnarExchange +Input [1]: [count#79] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(148) Exchange -Input [1]: [count#85] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] - -(149) HashAggregate [codegen id : 13] -Input [1]: [count#85] +(148) CometHashAggregate +Input [1]: [count#79] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#86] -Results [1]: [count(1)#86 AS h11_30_to_12#87] + +(149) ColumnarToRow [codegen id : 6] +Input [1]: [h11_30_to_12#80] (150) BroadcastExchange -Input [1]: [h11_30_to_12#87] +Input [1]: [h11_30_to_12#80] Arguments: IdentityBroadcastMode, [plan_id=13] -(151) BroadcastNestedLoopJoin [codegen id : 16] +(151) BroadcastNestedLoopJoin [codegen id : 8] Join type: Inner Join condition: None (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] +Output [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] 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 (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)) +Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] +Condition : ((isnotnull(ss_hdemo_sk#82) AND isnotnull(ss_sold_time_sk#81)) AND isnotnull(ss_store_sk#83)) (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] +Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] +Arguments: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83], [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83] (155) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#92] +Output [1]: [hd_demo_sk#85] (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 +Left output [3]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83] +Right output [1]: [hd_demo_sk#85] +Arguments: [ss_hdemo_sk#82], [hd_demo_sk#85], Inner, BuildRight (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] +Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, hd_demo_sk#85] +Arguments: [ss_sold_time_sk#81, ss_store_sk#83], [ss_sold_time_sk#81, ss_store_sk#83] (158) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#93, t_hour#94, t_minute#95] +Output [3]: [t_time_sk#86, t_hour#87, t_minute#88] 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 (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)) +Input [3]: [t_time_sk#86, t_hour#87, t_minute#88] +Condition : ((((isnotnull(t_hour#87) AND isnotnull(t_minute#88)) AND (t_hour#87 = 12)) AND (t_minute#88 < 30)) AND isnotnull(t_time_sk#86)) (160) CometProject -Input [3]: [t_time_sk#93, t_hour#94, t_minute#95] -Arguments: [t_time_sk#93], [t_time_sk#93] +Input [3]: [t_time_sk#86, t_hour#87, t_minute#88] +Arguments: [t_time_sk#86], [t_time_sk#86] (161) CometBroadcastExchange -Input [1]: [t_time_sk#93] -Arguments: [t_time_sk#93] +Input [1]: [t_time_sk#86] +Arguments: [t_time_sk#86] (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 +Left output [2]: [ss_sold_time_sk#81, ss_store_sk#83] +Right output [1]: [t_time_sk#86] +Arguments: [ss_sold_time_sk#81], [t_time_sk#86], Inner, BuildRight (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] +Input [3]: [ss_sold_time_sk#81, ss_store_sk#83, t_time_sk#86] +Arguments: [ss_store_sk#83], [ss_store_sk#83] (164) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#96] +Output [1]: [s_store_sk#89] (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 +Left output [1]: [ss_store_sk#83] +Right output [1]: [s_store_sk#89] +Arguments: [ss_store_sk#83], [s_store_sk#89], Inner, BuildRight (166) CometProject -Input [2]: [ss_store_sk#90, s_store_sk#96] +Input [2]: [ss_store_sk#83, s_store_sk#89] (167) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -(168) ColumnarToRow [codegen id : 14] -Input [1]: [count#97] +(168) CometColumnarExchange +Input [1]: [count#90] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] -(169) Exchange -Input [1]: [count#97] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=14] - -(170) HashAggregate [codegen id : 15] -Input [1]: [count#97] +(169) CometHashAggregate +Input [1]: [count#90] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#98] -Results [1]: [count(1)#98 AS h12_to_12_30#99] + +(170) ColumnarToRow [codegen id : 7] +Input [1]: [h12_to_12_30#91] (171) BroadcastExchange -Input [1]: [h12_to_12_30#99] +Input [1]: [h12_to_12_30#91] Arguments: IdentityBroadcastMode, [plan_id=15] -(172) BroadcastNestedLoopJoin [codegen id : 16] +(172) BroadcastNestedLoopJoin [codegen id : 8] Join type: Inner Join condition: None diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/simplified.txt index d3af10d19..b01685ed0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/simplified.txt @@ -1,4 +1,4 @@ -WholeStageCodegen (16) +WholeStageCodegen (8) BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin @@ -6,93 +6,39 @@ WholeStageCodegen (16) BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin - HashAggregate [count] [count(1),h8_30_to_9,count] + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] - CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_store_name] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] + CometHashAggregate [h8_30_to_9,count,count(1)] + CometColumnarExchange #1 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometBroadcastExchange [hd_demo_sk] #2 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange [t_time_sk] #3 + CometProject [t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + CometBroadcastExchange [s_store_sk] #4 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_store_name] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - HashAggregate [count] [count(1),h9_to_9_30,count] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange #6 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] - CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #7 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - HashAggregate [count] [count(1),h9_30_to_10,count] - InputAdapter - Exchange #9 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] - CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #10 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (7) - HashAggregate [count] [count(1),h10_to_10_30,count] - InputAdapter - Exchange #12 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter + CometHashAggregate [h9_to_9_30,count,count(1)] + CometColumnarExchange #6 CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] @@ -104,20 +50,18 @@ WholeStageCodegen (16) CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #13 + CometBroadcastExchange [t_time_sk] #7 CometProject [t_time_sk] CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (9) - HashAggregate [count] [count(1),h10_30_to_11,count] - InputAdapter - Exchange #15 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometHashAggregate [h9_30_to_10,count,count(1)] + CometColumnarExchange #9 CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] @@ -129,20 +73,18 @@ WholeStageCodegen (16) CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #16 + CometBroadcastExchange [t_time_sk] #10 CometProject [t_time_sk] CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (11) - HashAggregate [count] [count(1),h11_to_11_30,count] - InputAdapter - Exchange #18 - WholeStageCodegen (10) - ColumnarToRow - InputAdapter + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometHashAggregate [h10_to_10_30,count,count(1)] + CometColumnarExchange #12 CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] @@ -154,20 +96,18 @@ WholeStageCodegen (16) CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #19 + CometBroadcastExchange [t_time_sk] #13 CometProject [t_time_sk] CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #20 - WholeStageCodegen (13) - HashAggregate [count] [count(1),h11_30_to_12,count] - InputAdapter - Exchange #21 - WholeStageCodegen (12) - ColumnarToRow - InputAdapter + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometHashAggregate [h10_30_to_11,count,count(1)] + CometColumnarExchange #15 CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] @@ -179,20 +119,18 @@ WholeStageCodegen (16) CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #22 + CometBroadcastExchange [t_time_sk] #16 CometProject [t_time_sk] CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #23 - WholeStageCodegen (15) - HashAggregate [count] [count(1),h12_to_12_30,count] - InputAdapter - Exchange #24 - WholeStageCodegen (14) - ColumnarToRow - InputAdapter + InputAdapter + BroadcastExchange #17 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometHashAggregate [h11_to_11_30,count,count(1)] + CometColumnarExchange #18 CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] @@ -204,8 +142,54 @@ WholeStageCodegen (16) CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #25 + CometBroadcastExchange [t_time_sk] #19 CometProject [t_time_sk] CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 + InputAdapter + BroadcastExchange #20 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometHashAggregate [h11_30_to_12,count,count(1)] + CometColumnarExchange #21 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange [t_time_sk] #22 + CometProject [t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + ReusedExchange [s_store_sk] #4 + InputAdapter + BroadcastExchange #23 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometHashAggregate [h12_to_12_30,count,count(1)] + CometColumnarExchange #24 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange [t_time_sk] #25 + CometProject [t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] + 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-spark3_5/q89/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/explain.txt index 49dab66f0..6f973cee6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/explain.txt @@ -3,11 +3,11 @@ TakeOrderedAndProject (28) +- * Project (27) +- * Filter (26) +- Window (25) - +- * Sort (24) - +- Exchange (23) - +- * HashAggregate (22) - +- Exchange (21) - +- * ColumnarToRow (20) + +- * ColumnarToRow (24) + +- CometSort (23) + +- CometColumnarExchange (22) + +- CometHashAggregate (21) + +- CometColumnarExchange (20) +- CometHashAggregate (19) +- CometProject (18) +- CometBroadcastHashJoin (17) @@ -122,43 +122,41 @@ Input [7]: [i_brand#2, i_class#3, i_category#4, ss_sales_price#7, d_moy#12, s_st 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))] -(20) ColumnarToRow [codegen id : 1] +(20) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(22) HashAggregate [codegen id : 2] +(21) CometHashAggregate 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#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] -(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] +(22) CometColumnarExchange +Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#17, _w0#18] +Arguments: hashpartitioning(i_category#4, i_brand#2, s_store_name#14, s_company_name#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(23) CometSort +Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#17, _w0#18] +Arguments: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#17, _w0#18], [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] -(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) ColumnarToRow [codegen id : 1] +Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#17, _w0#18] (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] +Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#17, _w0#18] +Arguments: [avg(_w0#18) windowspecdefinition(i_category#4, i_brand#2, s_store_name#14, s_company_name#15, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#19], [i_category#4, i_brand#2, s_store_name#14, s_company_name#15] -(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) Filter [codegen id : 2] +Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#17, _w0#18, avg_monthly_sales#19] +Condition : CASE WHEN NOT (avg_monthly_sales#19 = 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#19)) / avg_monthly_sales#19) > 0.1000000000000000) END -(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) Project [codegen id : 2] +Output [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#17, avg_monthly_sales#19] +Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#17, _w0#18, avg_monthly_sales#19] (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] +Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#17, avg_monthly_sales#19] +Arguments: 100, [(sum_sales#17 - avg_monthly_sales#19) 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#17, avg_monthly_sales#19] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/simplified.txt index aa858c158..26c510f5a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/simplified.txt @@ -1,44 +1,40 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_class,i_brand,s_company_name,d_moy] - WholeStageCodegen (4) + WholeStageCodegen (2) 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 (3) - Sort [i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum,ss_sales_price] - CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] - CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometFilter [i_item_sk,i_brand,i_class,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #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] + CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 + CometHashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,sum,sum(UnscaledValue(ss_sales_price))] + CometColumnarExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 + CometHashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum,ss_sales_price] + CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] + CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_class,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_moy] #5 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/explain.txt index 1e97f8a15..375b036fc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/explain.txt @@ -26,10 +26,10 @@ Input [1]: [r_reason_sk#1] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#2, [id=#3] -* Project (12) -+- * HashAggregate (11) - +- Exchange (10) - +- * ColumnarToRow (9) +* ColumnarToRow (12) ++- CometProject (11) + +- CometHashAggregate (10) + +- CometColumnarExchange (9) +- CometHashAggregate (8) +- CometProject (7) +- CometFilter (6) @@ -56,33 +56,31 @@ Input [2]: [ss_ext_discount_amt#18, ss_net_paid#19] Keys: [] Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#18)), partial_avg(UnscaledValue(ss_net_paid#19))] -(9) ColumnarToRow [codegen id : 1] +(9) CometColumnarExchange Input [5]: [count#21, sum#22, count#23, sum#24, count#25] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(10) Exchange -Input [5]: [count#21, sum#22, count#23, sum#24, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] - -(11) HashAggregate [codegen id : 2] +(10) CometHashAggregate Input [5]: [count#21, sum#22, count#23, sum#24, count#25] Keys: [] Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#18)), avg(UnscaledValue(ss_net_paid#19))] -Aggregate Attributes [3]: [count(1)#26, avg(UnscaledValue(ss_ext_discount_amt#18))#27, avg(UnscaledValue(ss_net_paid#19))#28] -Results [3]: [count(1)#26 AS count(1)#29, cast((avg(UnscaledValue(ss_ext_discount_amt#18))#27 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#30, cast((avg(UnscaledValue(ss_net_paid#19))#28 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#31] -(12) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#29, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#30, avg(ss_net_paid), avg(ss_net_paid)#31) AS mergedValue#32] -Input [3]: [count(1)#29, avg(ss_ext_discount_amt)#30, avg(ss_net_paid)#31] +(11) CometProject +Input [3]: [count(1)#26, avg(ss_ext_discount_amt)#27, avg(ss_net_paid)#28] +Arguments: [mergedValue#29], [named_struct(count(1), count(1)#26, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#27, avg(ss_net_paid), avg(ss_net_paid)#28) AS mergedValue#29] + +(12) ColumnarToRow [codegen id : 1] +Input [1]: [mergedValue#29] Subquery:2 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] Subquery:3 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#5, [id=#6] -* Project (20) -+- * HashAggregate (19) - +- Exchange (18) - +- * ColumnarToRow (17) +* ColumnarToRow (20) ++- CometProject (19) + +- CometHashAggregate (18) + +- CometColumnarExchange (17) +- CometHashAggregate (16) +- CometProject (15) +- CometFilter (14) @@ -90,52 +88,50 @@ Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (13) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#33, ss_ext_discount_amt#34, ss_net_paid#35, ss_sold_date_sk#36] +Output [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct (14) CometFilter -Input [4]: [ss_quantity#33, ss_ext_discount_amt#34, ss_net_paid#35, ss_sold_date_sk#36] -Condition : ((isnotnull(ss_quantity#33) AND (ss_quantity#33 >= 21)) AND (ss_quantity#33 <= 40)) +Input [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] +Condition : ((isnotnull(ss_quantity#30) AND (ss_quantity#30 >= 21)) AND (ss_quantity#30 <= 40)) (15) CometProject -Input [4]: [ss_quantity#33, ss_ext_discount_amt#34, ss_net_paid#35, ss_sold_date_sk#36] -Arguments: [ss_ext_discount_amt#34, ss_net_paid#35], [ss_ext_discount_amt#34, ss_net_paid#35] +Input [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] +Arguments: [ss_ext_discount_amt#31, ss_net_paid#32], [ss_ext_discount_amt#31, ss_net_paid#32] (16) CometHashAggregate -Input [2]: [ss_ext_discount_amt#34, ss_net_paid#35] +Input [2]: [ss_ext_discount_amt#31, ss_net_paid#32] Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#34)), partial_avg(UnscaledValue(ss_net_paid#35))] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#31)), partial_avg(UnscaledValue(ss_net_paid#32))] -(17) ColumnarToRow [codegen id : 1] -Input [5]: [count#37, sum#38, count#39, sum#40, count#41] +(17) CometColumnarExchange +Input [5]: [count#34, sum#35, count#36, sum#37, count#38] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(18) Exchange -Input [5]: [count#37, sum#38, count#39, sum#40, count#41] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] - -(19) HashAggregate [codegen id : 2] -Input [5]: [count#37, sum#38, count#39, sum#40, count#41] +(18) CometHashAggregate +Input [5]: [count#34, sum#35, count#36, sum#37, count#38] Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#34)), avg(UnscaledValue(ss_net_paid#35))] -Aggregate Attributes [3]: [count(1)#42, avg(UnscaledValue(ss_ext_discount_amt#34))#43, avg(UnscaledValue(ss_net_paid#35))#44] -Results [3]: [count(1)#42 AS count(1)#45, cast((avg(UnscaledValue(ss_ext_discount_amt#34))#43 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#46, cast((avg(UnscaledValue(ss_net_paid#35))#44 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#47] +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#31)), avg(UnscaledValue(ss_net_paid#32))] + +(19) CometProject +Input [3]: [count(1)#39, avg(ss_ext_discount_amt)#40, avg(ss_net_paid)#41] +Arguments: [mergedValue#42], [named_struct(count(1), count(1)#39, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#40, avg(ss_net_paid), avg(ss_net_paid)#41) AS mergedValue#42] -(20) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#45, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#46, avg(ss_net_paid), avg(ss_net_paid)#47) AS mergedValue#48] -Input [3]: [count(1)#45, avg(ss_ext_discount_amt)#46, avg(ss_net_paid)#47] +(20) ColumnarToRow [codegen id : 1] +Input [1]: [mergedValue#42] Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] Subquery:6 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#8, [id=#9] -* Project (28) -+- * HashAggregate (27) - +- Exchange (26) - +- * ColumnarToRow (25) +* ColumnarToRow (28) ++- CometProject (27) + +- CometHashAggregate (26) + +- CometColumnarExchange (25) +- CometHashAggregate (24) +- CometProject (23) +- CometFilter (22) @@ -143,52 +139,50 @@ Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (21) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#49, ss_ext_discount_amt#50, ss_net_paid#51, ss_sold_date_sk#52] +Output [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct (22) CometFilter -Input [4]: [ss_quantity#49, ss_ext_discount_amt#50, ss_net_paid#51, ss_sold_date_sk#52] -Condition : ((isnotnull(ss_quantity#49) AND (ss_quantity#49 >= 41)) AND (ss_quantity#49 <= 60)) +Input [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] +Condition : ((isnotnull(ss_quantity#43) AND (ss_quantity#43 >= 41)) AND (ss_quantity#43 <= 60)) (23) CometProject -Input [4]: [ss_quantity#49, ss_ext_discount_amt#50, ss_net_paid#51, ss_sold_date_sk#52] -Arguments: [ss_ext_discount_amt#50, ss_net_paid#51], [ss_ext_discount_amt#50, ss_net_paid#51] +Input [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] +Arguments: [ss_ext_discount_amt#44, ss_net_paid#45], [ss_ext_discount_amt#44, ss_net_paid#45] (24) CometHashAggregate -Input [2]: [ss_ext_discount_amt#50, ss_net_paid#51] +Input [2]: [ss_ext_discount_amt#44, ss_net_paid#45] Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#50)), partial_avg(UnscaledValue(ss_net_paid#51))] - -(25) ColumnarToRow [codegen id : 1] -Input [5]: [count#53, sum#54, count#55, sum#56, count#57] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#44)), partial_avg(UnscaledValue(ss_net_paid#45))] -(26) Exchange -Input [5]: [count#53, sum#54, count#55, sum#56, count#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] +(25) CometColumnarExchange +Input [5]: [count#47, sum#48, count#49, sum#50, count#51] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(27) HashAggregate [codegen id : 2] -Input [5]: [count#53, sum#54, count#55, sum#56, count#57] +(26) CometHashAggregate +Input [5]: [count#47, sum#48, count#49, sum#50, count#51] Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#50)), avg(UnscaledValue(ss_net_paid#51))] -Aggregate Attributes [3]: [count(1)#58, avg(UnscaledValue(ss_ext_discount_amt#50))#59, avg(UnscaledValue(ss_net_paid#51))#60] -Results [3]: [count(1)#58 AS count(1)#61, cast((avg(UnscaledValue(ss_ext_discount_amt#50))#59 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#62, cast((avg(UnscaledValue(ss_net_paid#51))#60 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#63] +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#44)), avg(UnscaledValue(ss_net_paid#45))] -(28) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#61, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#62, avg(ss_net_paid), avg(ss_net_paid)#63) AS mergedValue#64] -Input [3]: [count(1)#61, avg(ss_ext_discount_amt)#62, avg(ss_net_paid)#63] +(27) CometProject +Input [3]: [count(1)#52, avg(ss_ext_discount_amt)#53, avg(ss_net_paid)#54] +Arguments: [mergedValue#55], [named_struct(count(1), count(1)#52, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#53, avg(ss_net_paid), avg(ss_net_paid)#54) AS mergedValue#55] + +(28) ColumnarToRow [codegen id : 1] +Input [1]: [mergedValue#55] Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] Subquery:9 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* Project (36) -+- * HashAggregate (35) - +- Exchange (34) - +- * ColumnarToRow (33) +* ColumnarToRow (36) ++- CometProject (35) + +- CometHashAggregate (34) + +- CometColumnarExchange (33) +- CometHashAggregate (32) +- CometProject (31) +- CometFilter (30) @@ -196,52 +190,50 @@ Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (29) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#65, ss_ext_discount_amt#66, ss_net_paid#67, ss_sold_date_sk#68] +Output [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct (30) CometFilter -Input [4]: [ss_quantity#65, ss_ext_discount_amt#66, ss_net_paid#67, ss_sold_date_sk#68] -Condition : ((isnotnull(ss_quantity#65) AND (ss_quantity#65 >= 61)) AND (ss_quantity#65 <= 80)) +Input [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] +Condition : ((isnotnull(ss_quantity#56) AND (ss_quantity#56 >= 61)) AND (ss_quantity#56 <= 80)) (31) CometProject -Input [4]: [ss_quantity#65, ss_ext_discount_amt#66, ss_net_paid#67, ss_sold_date_sk#68] -Arguments: [ss_ext_discount_amt#66, ss_net_paid#67], [ss_ext_discount_amt#66, ss_net_paid#67] +Input [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] +Arguments: [ss_ext_discount_amt#57, ss_net_paid#58], [ss_ext_discount_amt#57, ss_net_paid#58] (32) CometHashAggregate -Input [2]: [ss_ext_discount_amt#66, ss_net_paid#67] +Input [2]: [ss_ext_discount_amt#57, ss_net_paid#58] Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#66)), partial_avg(UnscaledValue(ss_net_paid#67))] - -(33) ColumnarToRow [codegen id : 1] -Input [5]: [count#69, sum#70, count#71, sum#72, count#73] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#57)), partial_avg(UnscaledValue(ss_net_paid#58))] -(34) Exchange -Input [5]: [count#69, sum#70, count#71, sum#72, count#73] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +(33) CometColumnarExchange +Input [5]: [count#60, sum#61, count#62, sum#63, count#64] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(35) HashAggregate [codegen id : 2] -Input [5]: [count#69, sum#70, count#71, sum#72, count#73] +(34) CometHashAggregate +Input [5]: [count#60, sum#61, count#62, sum#63, count#64] Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#66)), avg(UnscaledValue(ss_net_paid#67))] -Aggregate Attributes [3]: [count(1)#74, avg(UnscaledValue(ss_ext_discount_amt#66))#75, avg(UnscaledValue(ss_net_paid#67))#76] -Results [3]: [count(1)#74 AS count(1)#77, cast((avg(UnscaledValue(ss_ext_discount_amt#66))#75 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#78, cast((avg(UnscaledValue(ss_net_paid#67))#76 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#79] +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#57)), avg(UnscaledValue(ss_net_paid#58))] + +(35) CometProject +Input [3]: [count(1)#65, avg(ss_ext_discount_amt)#66, avg(ss_net_paid)#67] +Arguments: [mergedValue#68], [named_struct(count(1), count(1)#65, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#66, avg(ss_net_paid), avg(ss_net_paid)#67) AS mergedValue#68] -(36) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#77, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#78, avg(ss_net_paid), avg(ss_net_paid)#79) AS mergedValue#80] -Input [3]: [count(1)#77, avg(ss_ext_discount_amt)#78, avg(ss_net_paid)#79] +(36) ColumnarToRow [codegen id : 1] +Input [1]: [mergedValue#68] Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] Subquery:12 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#14, [id=#15] -* Project (44) -+- * HashAggregate (43) - +- Exchange (42) - +- * ColumnarToRow (41) +* ColumnarToRow (44) ++- CometProject (43) + +- CometHashAggregate (42) + +- CometColumnarExchange (41) +- CometHashAggregate (40) +- CometProject (39) +- CometFilter (38) @@ -249,42 +241,40 @@ Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (37) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#81, ss_ext_discount_amt#82, ss_net_paid#83, ss_sold_date_sk#84] +Output [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct (38) CometFilter -Input [4]: [ss_quantity#81, ss_ext_discount_amt#82, ss_net_paid#83, ss_sold_date_sk#84] -Condition : ((isnotnull(ss_quantity#81) AND (ss_quantity#81 >= 81)) AND (ss_quantity#81 <= 100)) +Input [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] +Condition : ((isnotnull(ss_quantity#69) AND (ss_quantity#69 >= 81)) AND (ss_quantity#69 <= 100)) (39) CometProject -Input [4]: [ss_quantity#81, ss_ext_discount_amt#82, ss_net_paid#83, ss_sold_date_sk#84] -Arguments: [ss_ext_discount_amt#82, ss_net_paid#83], [ss_ext_discount_amt#82, ss_net_paid#83] +Input [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] +Arguments: [ss_ext_discount_amt#70, ss_net_paid#71], [ss_ext_discount_amt#70, ss_net_paid#71] (40) CometHashAggregate -Input [2]: [ss_ext_discount_amt#82, ss_net_paid#83] +Input [2]: [ss_ext_discount_amt#70, ss_net_paid#71] Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#82)), partial_avg(UnscaledValue(ss_net_paid#83))] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#70)), partial_avg(UnscaledValue(ss_net_paid#71))] -(41) ColumnarToRow [codegen id : 1] -Input [5]: [count#85, sum#86, count#87, sum#88, count#89] +(41) CometColumnarExchange +Input [5]: [count#73, sum#74, count#75, sum#76, count#77] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(42) Exchange -Input [5]: [count#85, sum#86, count#87, sum#88, count#89] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5] - -(43) HashAggregate [codegen id : 2] -Input [5]: [count#85, sum#86, count#87, sum#88, count#89] +(42) CometHashAggregate +Input [5]: [count#73, sum#74, count#75, sum#76, count#77] Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#82)), avg(UnscaledValue(ss_net_paid#83))] -Aggregate Attributes [3]: [count(1)#90, avg(UnscaledValue(ss_ext_discount_amt#82))#91, avg(UnscaledValue(ss_net_paid#83))#92] -Results [3]: [count(1)#90 AS count(1)#93, cast((avg(UnscaledValue(ss_ext_discount_amt#82))#91 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#94, cast((avg(UnscaledValue(ss_net_paid#83))#92 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#95] +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#70)), avg(UnscaledValue(ss_net_paid#71))] + +(43) CometProject +Input [3]: [count(1)#78, avg(ss_ext_discount_amt)#79, avg(ss_net_paid)#80] +Arguments: [mergedValue#81], [named_struct(count(1), count(1)#78, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#79, avg(ss_net_paid), avg(ss_net_paid)#80) AS mergedValue#81] -(44) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#93, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#94, avg(ss_net_paid), avg(ss_net_paid)#95) AS mergedValue#96] -Input [3]: [count(1)#93, avg(ss_ext_discount_amt)#94, avg(ss_net_paid)#95] +(44) ColumnarToRow [codegen id : 1] +Input [1]: [mergedValue#81] Subquery:14 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/simplified.txt index 5cb600551..e0d03e2ef 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/simplified.txt @@ -1,78 +1,68 @@ WholeStageCodegen (1) Project Subquery #1 - WholeStageCodegen (2) - Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] - HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] - InputAdapter - Exchange #1 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] + CometHashAggregate [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count,count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] + CometColumnarExchange #1 + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] + CometProject [ss_ext_discount_amt,ss_net_paid] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #1 ReusedSubquery [mergedValue] #1 Subquery #2 - WholeStageCodegen (2) - Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] - HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] - InputAdapter - Exchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] + CometHashAggregate [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count,count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] + CometColumnarExchange #2 + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] + CometProject [ss_ext_discount_amt,ss_net_paid] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #2 ReusedSubquery [mergedValue] #2 Subquery #3 - WholeStageCodegen (2) - Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] - HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] - InputAdapter - Exchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] + CometHashAggregate [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count,count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] + CometColumnarExchange #3 + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] + CometProject [ss_ext_discount_amt,ss_net_paid] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #3 ReusedSubquery [mergedValue] #3 Subquery #4 - WholeStageCodegen (2) - Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] - HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] - InputAdapter - Exchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] + CometHashAggregate [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count,count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] + CometColumnarExchange #4 + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] + CometProject [ss_ext_discount_amt,ss_net_paid] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #4 ReusedSubquery [mergedValue] #4 Subquery #5 - WholeStageCodegen (2) - Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] - HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] - InputAdapter - Exchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] + CometHashAggregate [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count,count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] + CometColumnarExchange #5 + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] + CometProject [ss_ext_discount_amt,ss_net_paid] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #5 ReusedSubquery [mergedValue] #5 ColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/explain.txt index dbea5e75d..ab17ec1d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == * Project (47) +- * BroadcastNestedLoopJoin Inner BuildRight (46) - :- * HashAggregate (25) - : +- Exchange (24) - : +- * ColumnarToRow (23) + :- * ColumnarToRow (25) + : +- CometHashAggregate (24) + : +- CometColumnarExchange (23) : +- CometHashAggregate (22) : +- CometProject (21) : +- CometBroadcastHashJoin (20) @@ -27,9 +27,9 @@ : +- CometFilter (17) : +- CometScan parquet spark_catalog.default.web_page (16) +- BroadcastExchange (45) - +- * HashAggregate (44) - +- Exchange (43) - +- * ColumnarToRow (42) + +- * ColumnarToRow (44) + +- CometHashAggregate (43) + +- CometColumnarExchange (42) +- CometHashAggregate (41) +- CometProject (40) +- CometBroadcastHashJoin (39) @@ -151,114 +151,110 @@ Input: [] Keys: [] Functions [1]: [partial_count(1)] -(23) ColumnarToRow [codegen id : 1] +(23) CometColumnarExchange Input [1]: [count#11] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(24) Exchange -Input [1]: [count#11] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] - -(25) HashAggregate [codegen id : 4] +(24) CometHashAggregate Input [1]: [count#11] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#12] -Results [1]: [count(1)#12 AS amc#13] + +(25) ColumnarToRow [codegen id : 2] +Input [1]: [amc#12] (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] +Output [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] 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 (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)) +Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] +Condition : ((isnotnull(ws_ship_hdemo_sk#14) AND isnotnull(ws_sold_time_sk#13)) AND isnotnull(ws_web_page_sk#15)) (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] +Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] +Arguments: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15], [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15] (29) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#18] +Output [1]: [hd_demo_sk#17] (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 +Left output [3]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15] +Right output [1]: [hd_demo_sk#17] +Arguments: [ws_ship_hdemo_sk#14], [hd_demo_sk#17], Inner, BuildRight (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] +Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, hd_demo_sk#17] +Arguments: [ws_sold_time_sk#13, ws_web_page_sk#15], [ws_sold_time_sk#13, ws_web_page_sk#15] (32) Scan parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#19, t_hour#20] +Output [2]: [t_time_sk#18, t_hour#19] 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 (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)) +Input [2]: [t_time_sk#18, t_hour#19] +Condition : (((isnotnull(t_hour#19) AND (t_hour#19 >= 19)) AND (t_hour#19 <= 20)) AND isnotnull(t_time_sk#18)) (34) CometProject -Input [2]: [t_time_sk#19, t_hour#20] -Arguments: [t_time_sk#19], [t_time_sk#19] +Input [2]: [t_time_sk#18, t_hour#19] +Arguments: [t_time_sk#18], [t_time_sk#18] (35) CometBroadcastExchange -Input [1]: [t_time_sk#19] -Arguments: [t_time_sk#19] +Input [1]: [t_time_sk#18] +Arguments: [t_time_sk#18] (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 +Left output [2]: [ws_sold_time_sk#13, ws_web_page_sk#15] +Right output [1]: [t_time_sk#18] +Arguments: [ws_sold_time_sk#13], [t_time_sk#18], Inner, BuildRight (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] +Input [3]: [ws_sold_time_sk#13, ws_web_page_sk#15, t_time_sk#18] +Arguments: [ws_web_page_sk#15], [ws_web_page_sk#15] (38) ReusedExchange [Reuses operator id: 19] -Output [1]: [wp_web_page_sk#21] +Output [1]: [wp_web_page_sk#20] (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 +Left output [1]: [ws_web_page_sk#15] +Right output [1]: [wp_web_page_sk#20] +Arguments: [ws_web_page_sk#15], [wp_web_page_sk#20], Inner, BuildRight (40) CometProject -Input [2]: [ws_web_page_sk#16, wp_web_page_sk#21] +Input [2]: [ws_web_page_sk#15, wp_web_page_sk#20] (41) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -(42) ColumnarToRow [codegen id : 2] -Input [1]: [count#22] +(42) CometColumnarExchange +Input [1]: [count#21] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(43) Exchange -Input [1]: [count#22] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] - -(44) HashAggregate [codegen id : 3] -Input [1]: [count#22] +(43) CometHashAggregate +Input [1]: [count#21] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#23] -Results [1]: [count(1)#23 AS pmc#24] + +(44) ColumnarToRow [codegen id : 1] +Input [1]: [pmc#22] (45) BroadcastExchange -Input [1]: [pmc#24] +Input [1]: [pmc#22] Arguments: IdentityBroadcastMode, [plan_id=3] -(46) BroadcastNestedLoopJoin [codegen id : 4] +(46) BroadcastNestedLoopJoin [codegen id : 2] Join type: Inner Join condition: None -(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] +(47) Project [codegen id : 2] +Output [1]: [(cast(amc#12 as decimal(15,4)) / cast(pmc#22 as decimal(15,4))) AS am_pm_ratio#23] +Input [2]: [amc#12, pmc#22] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/simplified.txt index 44159cc73..65860a550 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/simplified.txt @@ -1,56 +1,52 @@ -WholeStageCodegen (4) +WholeStageCodegen (2) Project [amc,pmc] BroadcastNestedLoopJoin - HashAggregate [count] [count(1),amc,count] + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] - CometProject [ws_sold_time_sk,ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] - CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - CometBroadcastExchange [wp_web_page_sk] #4 - CometProject [wp_web_page_sk] - CometFilter [wp_web_page_sk,wp_char_count] - CometScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] + CometHashAggregate [amc,count,count(1)] + CometColumnarExchange #1 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometProject [ws_web_page_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] + CometProject [ws_sold_time_sk,ws_web_page_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] + CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] + CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] + CometBroadcastExchange [hd_demo_sk] #2 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] + CometBroadcastExchange [t_time_sk] #3 + CometProject [t_time_sk] + CometFilter [t_time_sk,t_hour] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] + CometBroadcastExchange [wp_web_page_sk] #4 + CometProject [wp_web_page_sk] + CometFilter [wp_web_page_sk,wp_char_count] + CometScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - HashAggregate [count] [count(1),pmc,count] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange #6 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] - CometProject [ws_sold_time_sk,ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] - CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #7 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - ReusedExchange [wp_web_page_sk] #4 + CometHashAggregate [pmc,count,count(1)] + CometColumnarExchange #6 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometProject [ws_web_page_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] + CometProject [ws_sold_time_sk,ws_web_page_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] + CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] + CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange [t_time_sk] #7 + CometProject [t_time_sk] + CometFilter [t_time_sk,t_hour] + 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-spark3_5/q91/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/explain.txt index cd2e8cfb0..836e03f4a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -* Sort (41) -+- Exchange (40) - +- * HashAggregate (39) - +- Exchange (38) - +- * ColumnarToRow (37) +* ColumnarToRow (41) ++- CometSort (40) + +- CometColumnarExchange (39) + +- CometHashAggregate (38) + +- CometColumnarExchange (37) +- CometHashAggregate (36) +- CometProject (35) +- CometBroadcastHashJoin (34) @@ -215,27 +215,25 @@ Input [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, cd_mari 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))] -(37) ColumnarToRow [codegen id : 1] +(37) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(39) HashAggregate [codegen id : 2] +(38) CometHashAggregate 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))#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] -(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] +(39) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] + +(40) CometSort +Input [4]: [Call_Center#25, Call_Center_Name#26, Manager#27, Returns_Loss#28] +Arguments: [Call_Center#25, Call_Center_Name#26, Manager#27, Returns_Loss#28], [Returns_Loss#28 DESC NULLS LAST] -(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 +(41) ColumnarToRow [codegen id : 1] +Input [4]: [Call_Center#25, Call_Center_Name#26, Manager#27, Returns_Loss#28] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/simplified.txt index 232c174cd..ff17b986a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/simplified.txt @@ -1,55 +1,51 @@ -WholeStageCodegen (3) - Sort [Returns_Loss] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [Returns_Loss] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum,cr_net_loss] - CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status,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 [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status] - CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk] - CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,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 [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss] - CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,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,cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] - CometFilter [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometBroadcastExchange [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] #3 - CometFilter [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] #6 - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk] #7 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #8 - 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 [hd_demo_sk] #9 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] + CometSort [Call_Center,Call_Center_Name,Manager,Returns_Loss] + CometColumnarExchange [Returns_Loss] #1 + CometHashAggregate [Call_Center,Call_Center_Name,Manager,Returns_Loss,cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum,sum(UnscaledValue(cr_net_loss))] + CometColumnarExchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 + CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum,cr_net_loss] + CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status,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 [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status] + CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,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 [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,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,cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] + CometFilter [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] + CometBroadcastExchange [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] #3 + CometFilter [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_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_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] #6 + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + CometBroadcastExchange [ca_address_sk] #7 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #8 + 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 [hd_demo_sk] #9 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] + 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-spark3_5/q92/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/explain.txt index 074fe2b99..7d3423272 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/explain.txt @@ -1,35 +1,34 @@ == Physical Plan == -* 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) +* ColumnarToRow (30) ++- CometHashAggregate (29) + +- CometColumnarExchange (28) + +- CometHashAggregate (27) + +- CometProject (26) + +- CometBroadcastHashJoin (25) + :- CometProject (23) + : +- CometBroadcastHashJoin (22) + : :- 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) + : +- CometBroadcastExchange (21) + : +- CometFilter (20) + : +- CometHashAggregate (19) + : +- CometColumnarExchange (18) + : +- CometHashAggregate (17) + : +- CometProject (16) + : +- CometBroadcastHashJoin (15) + : :- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.web_sales (9) + : +- CometBroadcastExchange (14) + : +- CometProject (13) + : +- CometFilter (12) + : +- CometScan parquet spark_catalog.default.date_dim (11) + +- ReusedExchange (24) (1) Scan parquet spark_catalog.default.web_sales @@ -72,10 +71,7 @@ Arguments: [ws_item_sk#1], [i_item_sk#5], Inner, BuildRight 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] - -(10) Scan parquet spark_catalog.default.web_sales +(9) 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 [] @@ -83,138 +79,130 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(11) CometFilter +(10) CometFilter Input [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] Condition : isnotnull(ws_item_sk#7) -(12) Scan parquet spark_catalog.default.date_dim +(11) 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 -(13) CometFilter +(12) 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)) -(14) CometProject +(13) CometProject Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(15) CometBroadcastExchange +(14) CometBroadcastExchange Input [1]: [d_date_sk#11] Arguments: [d_date_sk#11] -(16) CometBroadcastHashJoin +(15) 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 +(16) 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) CometHashAggregate +(17) 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))] -(19) ColumnarToRow [codegen id : 1] -Input [3]: [ws_item_sk#7, sum#13, count#14] - -(20) Exchange +(18) CometColumnarExchange Input [3]: [ws_item_sk#7, sum#13, count#14] -Arguments: hashpartitioning(ws_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: hashpartitioning(ws_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(21) HashAggregate [codegen id : 2] +(19) CometHashAggregate 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))#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] -(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) +(20) CometFilter +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] +Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#15) -(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] +(21) CometBroadcastExchange +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] +Arguments: [(1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] -(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))#16) +(22) CometBroadcastHashJoin +Left output [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] +Right output [2]: [(1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] +Arguments: [i_item_sk#5], [ws_item_sk#7], Inner, (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#15), BuildRight -(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))#16, ws_item_sk#7] +(23) CometProject +Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] +Arguments: [ws_ext_discount_amt#2, ws_sold_date_sk#3], [ws_ext_discount_amt#2, ws_sold_date_sk#3] -(26) ReusedExchange [Reuses operator id: 36] -Output [1]: [d_date_sk#17] +(24) ReusedExchange [Reuses operator id: 14] +Output [1]: [d_date_sk#16] -(27) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#17] -Join type: Inner -Join condition: None +(25) CometBroadcastHashJoin +Left output [2]: [ws_ext_discount_amt#2, ws_sold_date_sk#3] +Right output [1]: [d_date_sk#16] +Arguments: [ws_sold_date_sk#3], [d_date_sk#16], Inner, BuildRight -(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#17] +(26) CometProject +Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#16] +Arguments: [ws_ext_discount_amt#2], [ws_ext_discount_amt#2] -(29) HashAggregate [codegen id : 4] +(27) CometHashAggregate Input [1]: [ws_ext_discount_amt#2] Keys: [] Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum#18] -Results [1]: [sum#19] -(30) Exchange -Input [1]: [sum#19] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] +(28) CometColumnarExchange +Input [1]: [sum#17] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(31) HashAggregate [codegen id : 5] -Input [1]: [sum#19] +(29) CometHashAggregate +Input [1]: [sum#17] Keys: [] Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] -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] + +(30) ColumnarToRow [codegen id : 1] +Input [1]: [Excess Discount Amount #18] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (36) -+- * ColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan parquet spark_catalog.default.date_dim (32) +BroadcastExchange (35) ++- * ColumnarToRow (34) + +- CometProject (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.date_dim (31) -(32) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#17, d_date#22] +(31) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#16, d_date#19] 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 -(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) CometFilter +Input [2]: [d_date_sk#16, d_date#19] +Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-01-27)) AND (d_date#19 <= 2000-04-26)) AND isnotnull(d_date_sk#16)) -(34) CometProject -Input [2]: [d_date_sk#17, d_date#22] -Arguments: [d_date_sk#17], [d_date_sk#17] +(33) CometProject +Input [2]: [d_date_sk#16, d_date#19] +Arguments: [d_date_sk#16], [d_date_sk#16] -(35) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#17] +(34) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#16] -(36) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(35) BroadcastExchange +Input [1]: [d_date_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -Subquery:2 Hosting operator id = 10 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 9 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/simplified.txt index 0681a64bd..065e38bb1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/simplified.txt @@ -1,50 +1,41 @@ -WholeStageCodegen (5) - HashAggregate [sum] [sum(UnscaledValue(ws_ext_discount_amt)),Excess Discount Amount ,sum] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange #1 - 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))] - ColumnarToRow - InputAdapter - CometProject [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] - CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] - CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk] #3 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - InputAdapter - BroadcastExchange #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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ws_item_sk,sum,count,ws_ext_discount_amt] - CometProject [ws_item_sk,ws_ext_discount_amt] - CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - ReusedExchange [d_date_sk] #2 + CometHashAggregate [Excess Discount Amount ,sum,sum(UnscaledValue(ws_ext_discount_amt))] + CometColumnarExchange #1 + CometHashAggregate [sum,ws_ext_discount_amt] + CometProject [ws_ext_discount_amt] + CometBroadcastHashJoin [ws_ext_discount_amt,ws_sold_date_sk,d_date_sk] + CometProject [ws_ext_discount_amt,ws_sold_date_sk] + CometBroadcastHashJoin [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk,(1.3 * avg(ws_ext_discount_amt)),ws_item_sk] + CometProject [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] + CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk] #3 + CometProject [i_item_sk] + CometFilter [i_item_sk,i_manufact_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + CometBroadcastExchange [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk] #4 + CometFilter [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk] + CometHashAggregate [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk,sum,count,avg(UnscaledValue(ws_ext_discount_amt))] + CometColumnarExchange [ws_item_sk] #5 + CometHashAggregate [ws_item_sk,sum,count,ws_ext_discount_amt] + CometProject [ws_item_sk,ws_ext_discount_amt] + CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/explain.txt index 00ed822f2..ee3fb07aa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/explain.txt @@ -1,28 +1,26 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * HashAggregate (23) - +- Exchange (22) - +- * HashAggregate (21) - +- * Project (20) - +- * BroadcastHashJoin Inner BuildRight (19) - :- * Project (13) - : +- * SortMergeJoin Inner (12) - : :- * Sort (5) - : : +- Exchange (4) - : : +- * ColumnarToRow (3) - : : +- CometProject (2) - : : +- CometScan parquet spark_catalog.default.store_sales (1) - : +- * Sort (11) - : +- Exchange (10) - : +- * ColumnarToRow (9) - : +- CometProject (8) - : +- CometFilter (7) - : +- CometScan parquet spark_catalog.default.store_returns (6) - +- BroadcastExchange (18) - +- * ColumnarToRow (17) - +- CometProject (16) - +- CometFilter (15) - +- CometScan parquet spark_catalog.default.reason (14) +* ColumnarToRow (22) ++- CometTakeOrderedAndProject (21) + +- CometHashAggregate (20) + +- CometColumnarExchange (19) + +- CometHashAggregate (18) + +- CometProject (17) + +- CometBroadcastHashJoin (16) + :- CometProject (11) + : +- CometSortMergeJoin (10) + : :- CometSort (4) + : : +- CometColumnarExchange (3) + : : +- CometProject (2) + : : +- CometScan parquet spark_catalog.default.store_sales (1) + : +- CometSort (9) + : +- CometColumnarExchange (8) + : +- CometProject (7) + : +- CometFilter (6) + : +- CometScan parquet spark_catalog.default.store_returns (5) + +- CometBroadcastExchange (15) + +- CometProject (14) + +- CometFilter (13) + +- CometScan parquet spark_catalog.default.reason (12) (1) Scan parquet spark_catalog.default.store_sales @@ -35,104 +33,92 @@ ReadSchema: struct -(7) CometFilter +(6) CometFilter Input [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] Condition : ((isnotnull(sr_item_sk#7) AND isnotnull(sr_ticket_number#9)) AND isnotnull(sr_reason_sk#8)) -(8) CometProject +(7) CometProject Input [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] Arguments: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10], [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -(9) ColumnarToRow [codegen id : 3] -Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] - -(10) Exchange +(8) CometColumnarExchange Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: hashpartitioning(sr_item_sk#7, sr_ticket_number#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(sr_item_sk#7, sr_ticket_number#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(11) Sort [codegen id : 4] +(9) CometSort Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [sr_item_sk#7 ASC NULLS FIRST, sr_ticket_number#9 ASC NULLS FIRST], false, 0 +Arguments: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10], [sr_item_sk#7 ASC NULLS FIRST, sr_ticket_number#9 ASC NULLS FIRST] -(12) SortMergeJoin [codegen id : 6] -Left keys [2]: [ss_item_sk#1, ss_ticket_number#3] -Right keys [2]: [sr_item_sk#7, sr_ticket_number#9] -Join type: Inner -Join condition: None +(10) CometSortMergeJoin +Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] +Right output [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] +Arguments: [ss_item_sk#1, ss_ticket_number#3], [sr_item_sk#7, sr_ticket_number#9], Inner -(13) Project [codegen id : 6] -Output [5]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10] +(11) CometProject Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] +Arguments: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10], [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10] -(14) Scan parquet spark_catalog.default.reason +(12) Scan parquet spark_catalog.default.reason Output [2]: [r_reason_sk#12, r_reason_desc#13] Batched: true Location [not included in comparison]/{warehouse_dir}/reason] PushedFilters: [IsNotNull(r_reason_desc), EqualTo(r_reason_desc,reason 28 ), IsNotNull(r_reason_sk)] ReadSchema: struct -(15) CometFilter +(13) CometFilter Input [2]: [r_reason_sk#12, r_reason_desc#13] Condition : ((isnotnull(r_reason_desc#13) AND (r_reason_desc#13 = reason 28 )) AND isnotnull(r_reason_sk#12)) -(16) CometProject +(14) CometProject Input [2]: [r_reason_sk#12, r_reason_desc#13] Arguments: [r_reason_sk#12], [r_reason_sk#12] -(17) ColumnarToRow [codegen id : 5] +(15) CometBroadcastExchange Input [1]: [r_reason_sk#12] +Arguments: [r_reason_sk#12] -(18) BroadcastExchange -Input [1]: [r_reason_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(19) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_reason_sk#8] -Right keys [1]: [r_reason_sk#12] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [5]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10] +Right output [1]: [r_reason_sk#12] +Arguments: [sr_reason_sk#8], [r_reason_sk#12], Inner, BuildRight -(20) Project [codegen id : 6] -Output [2]: [ss_customer_sk#2, CASE WHEN isnotnull(sr_return_quantity#10) THEN (cast((ss_quantity#4 - sr_return_quantity#10) as decimal(10,0)) * ss_sales_price#5) ELSE (cast(ss_quantity#4 as decimal(10,0)) * ss_sales_price#5) END AS act_sales#14] +(17) CometProject Input [6]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10, r_reason_sk#12] +Arguments: [ss_customer_sk#2, act_sales#14], [ss_customer_sk#2, CASE WHEN isnotnull(sr_return_quantity#10) THEN (cast((ss_quantity#4 - sr_return_quantity#10) as decimal(10,0)) * ss_sales_price#5) ELSE (cast(ss_quantity#4 as decimal(10,0)) * ss_sales_price#5) END AS act_sales#14] -(21) HashAggregate [codegen id : 6] +(18) CometHashAggregate Input [2]: [ss_customer_sk#2, act_sales#14] Keys [1]: [ss_customer_sk#2] Functions [1]: [partial_sum(act_sales#14)] -Aggregate Attributes [2]: [sum#15, isEmpty#16] -Results [3]: [ss_customer_sk#2, sum#17, isEmpty#18] -(22) Exchange -Input [3]: [ss_customer_sk#2, sum#17, isEmpty#18] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(19) CometColumnarExchange +Input [3]: [ss_customer_sk#2, sum#15, isEmpty#16] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) HashAggregate [codegen id : 7] -Input [3]: [ss_customer_sk#2, sum#17, isEmpty#18] +(20) CometHashAggregate +Input [3]: [ss_customer_sk#2, sum#15, isEmpty#16] Keys [1]: [ss_customer_sk#2] Functions [1]: [sum(act_sales#14)] -Aggregate Attributes [1]: [sum(act_sales#14)#19] -Results [2]: [ss_customer_sk#2, sum(act_sales#14)#19 AS sumsales#20] -(24) TakeOrderedAndProject -Input [2]: [ss_customer_sk#2, sumsales#20] -Arguments: 100, [sumsales#20 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], [ss_customer_sk#2, sumsales#20] +(21) CometTakeOrderedAndProject +Input [2]: [ss_customer_sk#2, sumsales#17] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[sumsales#17 ASC NULLS FIRST,ss_customer_sk#2 ASC NULLS FIRST], output=[ss_customer_sk#2,sumsales#17]), [ss_customer_sk#2, sumsales#17], 100, [sumsales#17 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], [ss_customer_sk#2, sumsales#17] + +(22) ColumnarToRow [codegen id : 1] +Input [2]: [ss_customer_sk#2, sumsales#17] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/simplified.txt index a8eb231b7..c4fb926a5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/simplified.txt @@ -1,40 +1,24 @@ -TakeOrderedAndProject [sumsales,ss_customer_sk] - WholeStageCodegen (7) - HashAggregate [ss_customer_sk,sum,isEmpty] [sum(act_sales),sumsales,sum,isEmpty] - InputAdapter - Exchange [ss_customer_sk] #1 - WholeStageCodegen (6) - HashAggregate [ss_customer_sk,act_sales] [sum,isEmpty,sum,isEmpty] - Project [ss_customer_sk,sr_return_quantity,ss_quantity,ss_sales_price] - BroadcastHashJoin [sr_reason_sk,r_reason_sk] - Project [ss_customer_sk,ss_quantity,ss_sales_price,sr_reason_sk,sr_return_quantity] - SortMergeJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - WholeStageCodegen (2) - Sort [ss_item_sk,ss_ticket_number] - InputAdapter - Exchange [ss_item_sk,ss_ticket_number] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] - InputAdapter - WholeStageCodegen (4) - Sort [sr_item_sk,sr_ticket_number] - InputAdapter - Exchange [sr_item_sk,sr_ticket_number] #3 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometFilter [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometProject [r_reason_sk] - CometFilter [r_reason_sk,r_reason_desc] - CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [ss_customer_sk,sumsales] + CometHashAggregate [ss_customer_sk,sumsales,sum,isEmpty,sum(act_sales)] + CometColumnarExchange [ss_customer_sk] #1 + CometHashAggregate [ss_customer_sk,sum,isEmpty,act_sales] + CometProject [sr_return_quantity,ss_quantity,ss_sales_price] [ss_customer_sk,act_sales] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,sr_reason_sk,sr_return_quantity,r_reason_sk] + CometProject [ss_customer_sk,ss_quantity,ss_sales_price,sr_reason_sk,sr_return_quantity] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] + CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] + CometColumnarExchange [ss_item_sk,ss_ticket_number] #2 + CometProject [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometSort [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] + CometColumnarExchange [sr_item_sk,sr_ticket_number] #3 + CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] + CometFilter [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometBroadcastExchange [r_reason_sk] #4 + CometProject [r_reason_sk] + CometFilter [r_reason_sk,r_reason_desc] + 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-spark3_5/q94/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/explain.txt index d71f96e15..ff1dd24d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/explain.txt @@ -1,49 +1,51 @@ == Physical Plan == -* HashAggregate (45) -+- Exchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * SortMergeJoin LeftAnti (19) - : : : :- * Project (13) - : : : : +- * SortMergeJoin LeftSemi (12) - : : : : :- * Sort (6) - : : : : : +- Exchange (5) - : : : : : +- * ColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : : +- * Sort (11) - : : : : +- Exchange (10) - : : : : +- * ColumnarToRow (9) - : : : : +- CometProject (8) - : : : : +- CometScan parquet spark_catalog.default.web_sales (7) - : : : +- * Sort (18) - : : : +- Exchange (17) - : : : +- * ColumnarToRow (16) - : : : +- CometProject (15) - : : : +- CometScan parquet spark_catalog.default.web_returns (14) - : : +- BroadcastExchange (24) - : : +- * ColumnarToRow (23) - : : +- CometProject (22) - : : +- CometFilter (21) - : : +- CometScan parquet spark_catalog.default.date_dim (20) - : +- 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.web_site (34) +* HashAggregate (47) ++- * ColumnarToRow (46) + +- CometColumnarExchange (45) + +- RowToColumnar (44) + +- * HashAggregate (43) + +- * HashAggregate (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * SortMergeJoin LeftAnti (19) + : : : :- * Project (13) + : : : : +- * SortMergeJoin LeftSemi (12) + : : : : :- * ColumnarToRow (6) + : : : : : +- CometSort (5) + : : : : : +- CometColumnarExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : : +- * ColumnarToRow (11) + : : : : +- CometSort (10) + : : : : +- CometColumnarExchange (9) + : : : : +- CometProject (8) + : : : : +- CometScan parquet spark_catalog.default.web_sales (7) + : : : +- * ColumnarToRow (18) + : : : +- CometSort (17) + : : : +- CometColumnarExchange (16) + : : : +- CometProject (15) + : : : +- CometScan parquet spark_catalog.default.web_returns (14) + : : +- BroadcastExchange (24) + : : +- * ColumnarToRow (23) + : : +- CometProject (22) + : : +- CometFilter (21) + : : +- CometScan parquet spark_catalog.default.date_dim (20) + : +- 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.web_site (34) (1) Scan parquet spark_catalog.default.web_sales @@ -61,16 +63,16 @@ Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -(4) ColumnarToRow [codegen id : 1] +(4) CometColumnarExchange Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] +Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(5) Exchange +(5) CometSort Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_order_number#5 ASC NULLS FIRST] -(6) Sort [codegen id : 2] +(6) ColumnarToRow [codegen id : 1] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: [ws_order_number#5 ASC NULLS FIRST], false, 0 (7) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] @@ -82,24 +84,24 @@ ReadSchema: struct Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] Arguments: [ws_warehouse_sk#9, ws_order_number#10], [ws_warehouse_sk#9, ws_order_number#10] -(9) ColumnarToRow [codegen id : 3] +(9) CometColumnarExchange Input [2]: [ws_warehouse_sk#9, ws_order_number#10] +Arguments: hashpartitioning(ws_order_number#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(10) Exchange +(10) CometSort Input [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: hashpartitioning(ws_order_number#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: [ws_warehouse_sk#9, ws_order_number#10], [ws_order_number#10 ASC NULLS FIRST] -(11) Sort [codegen id : 4] +(11) ColumnarToRow [codegen id : 2] Input [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: [ws_order_number#10 ASC NULLS FIRST], false, 0 -(12) SortMergeJoin [codegen id : 5] +(12) SortMergeJoin [codegen id : 3] Left keys [1]: [ws_order_number#5] Right keys [1]: [ws_order_number#10] Join type: LeftSemi Join condition: NOT (ws_warehouse_sk#4 = ws_warehouse_sk#9) -(13) Project [codegen id : 5] +(13) Project [codegen id : 3] Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] @@ -113,18 +115,18 @@ ReadSchema: struct Input [2]: [wr_order_number#12, wr_returned_date_sk#13] Arguments: [wr_order_number#12], [wr_order_number#12] -(16) ColumnarToRow [codegen id : 6] +(16) CometColumnarExchange Input [1]: [wr_order_number#12] +Arguments: hashpartitioning(wr_order_number#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(17) Exchange +(17) CometSort Input [1]: [wr_order_number#12] -Arguments: hashpartitioning(wr_order_number#12, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: [wr_order_number#12], [wr_order_number#12 ASC NULLS FIRST] -(18) Sort [codegen id : 7] +(18) ColumnarToRow [codegen id : 4] Input [1]: [wr_order_number#12] -Arguments: [wr_order_number#12 ASC NULLS FIRST], false, 0 -(19) SortMergeJoin [codegen id : 11] +(19) SortMergeJoin [codegen id : 8] Left keys [1]: [ws_order_number#5] Right keys [1]: [wr_order_number#12] Join type: LeftAnti @@ -145,20 +147,20 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-01)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(23) ColumnarToRow [codegen id : 8] +(23) ColumnarToRow [codegen id : 5] Input [1]: [d_date_sk#14] (24) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(25) BroadcastHashJoin [codegen id : 11] +(25) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ws_ship_date_sk#1] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(26) Project [codegen id : 11] +(26) Project [codegen id : 8] Output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#14] @@ -177,20 +179,20 @@ Condition : ((isnotnull(ca_state#17) AND (ca_state#17 = IL)) AND isnotnull(ca_ad Input [2]: [ca_address_sk#16, ca_state#17] Arguments: [ca_address_sk#16], [ca_address_sk#16] -(30) ColumnarToRow [codegen id : 9] +(30) ColumnarToRow [codegen id : 6] Input [1]: [ca_address_sk#16] (31) BroadcastExchange Input [1]: [ca_address_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(32) BroadcastHashJoin [codegen id : 11] +(32) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ws_ship_addr_sk#2] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(33) Project [codegen id : 11] +(33) Project [codegen id : 8] Output [4]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#16] @@ -209,49 +211,55 @@ Condition : ((isnotnull(web_company_name#19) AND (web_company_name#19 = pri Input [2]: [web_site_sk#18, web_company_name#19] Arguments: [web_site_sk#18], [web_site_sk#18] -(37) ColumnarToRow [codegen id : 10] +(37) ColumnarToRow [codegen id : 7] Input [1]: [web_site_sk#18] (38) BroadcastExchange Input [1]: [web_site_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(39) BroadcastHashJoin [codegen id : 11] +(39) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ws_web_site_sk#3] Right keys [1]: [web_site_sk#18] Join type: Inner Join condition: None -(40) Project [codegen id : 11] +(40) Project [codegen id : 8] Output [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [5]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#18] -(41) HashAggregate [codegen id : 11] +(41) HashAggregate [codegen id : 8] Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Keys [1]: [ws_order_number#5] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#20, sum(UnscaledValue(ws_net_profit#7))#21] Results [3]: [ws_order_number#5, sum#22, sum#23] -(42) HashAggregate [codegen id : 11] +(42) HashAggregate [codegen id : 8] Input [3]: [ws_order_number#5, sum#22, sum#23] Keys [1]: [ws_order_number#5] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#20, sum(UnscaledValue(ws_net_profit#7))#21] Results [3]: [ws_order_number#5, sum#22, sum#23] -(43) HashAggregate [codegen id : 11] +(43) HashAggregate [codegen id : 8] Input [3]: [ws_order_number#5, sum#22, sum#23] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#20, sum(UnscaledValue(ws_net_profit#7))#21, count(ws_order_number#5)#24] Results [3]: [sum#22, sum#23, count#25] -(44) Exchange +(44) RowToColumnar Input [3]: [sum#22, sum#23, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(45) HashAggregate [codegen id : 12] +(45) CometColumnarExchange +Input [3]: [sum#22, sum#23, count#25] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(46) ColumnarToRow [codegen id : 9] +Input [3]: [sum#22, sum#23, count#25] + +(47) HashAggregate [codegen id : 9] Input [3]: [sum#22, sum#23, count#25] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/simplified.txt index d54c9e0c9..3f324f9f2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/simplified.txt @@ -1,74 +1,70 @@ -WholeStageCodegen (12) +WholeStageCodegen (9) HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] - InputAdapter - Exchange #1 - WholeStageCodegen (11) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] - Project [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_date_sk,d_date_sk] - SortMergeJoin [ws_order_number,wr_order_number] - InputAdapter - WholeStageCodegen (5) - Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] - InputAdapter - WholeStageCodegen (2) - Sort [ws_order_number] - InputAdapter - Exchange [ws_order_number] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometColumnarExchange #1 + RowToColumnar + WholeStageCodegen (8) + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] + Project [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_date_sk,d_date_sk] + SortMergeJoin [ws_order_number,wr_order_number] + InputAdapter + WholeStageCodegen (3) + Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometSort [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometColumnarExchange [ws_order_number] #2 CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (4) - Sort [ws_order_number] - InputAdapter - Exchange [ws_order_number] #3 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter + InputAdapter + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometSort [ws_warehouse_sk,ws_order_number] + CometColumnarExchange [ws_order_number] #3 CometProject [ws_warehouse_sk,ws_order_number] CometScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (7) - Sort [wr_order_number] - InputAdapter - Exchange [wr_order_number] #4 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter + InputAdapter + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometSort [wr_order_number] + CometColumnarExchange [wr_order_number] #4 CometProject [wr_order_number] CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (8) + BroadcastExchange #6 + WholeStageCodegen (6) ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) + BroadcastExchange #7 + WholeStageCodegen (7) ColumnarToRow InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometProject [web_site_sk] - CometFilter [web_site_sk,web_company_name] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] + CometProject [web_site_sk] + CometFilter [web_site_sk,web_company_name] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/explain.txt index c8cdce055..f58d42056 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/explain.txt @@ -1,62 +1,67 @@ == Physical Plan == -* HashAggregate (58) -+- Exchange (57) - +- * HashAggregate (56) - +- * HashAggregate (55) - +- * HashAggregate (54) - +- * Project (53) - +- * BroadcastHashJoin Inner BuildRight (52) - :- * Project (46) - : +- * BroadcastHashJoin Inner BuildRight (45) - : :- * Project (39) - : : +- * BroadcastHashJoin Inner BuildRight (38) - : : :- * SortMergeJoin LeftSemi (32) - : : : :- * SortMergeJoin LeftSemi (17) - : : : : :- * Sort (6) - : : : : : +- Exchange (5) - : : : : : +- * ColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : : +- * Project (16) - : : : : +- * SortMergeJoin Inner (15) - : : : : :- * Sort (12) - : : : : : +- Exchange (11) - : : : : : +- * ColumnarToRow (10) - : : : : : +- CometProject (9) - : : : : : +- CometFilter (8) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (7) - : : : : +- * Sort (14) - : : : : +- ReusedExchange (13) - : : : +- * Project (31) - : : : +- * SortMergeJoin Inner (30) - : : : :- * Sort (23) - : : : : +- Exchange (22) - : : : : +- * ColumnarToRow (21) - : : : : +- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan parquet spark_catalog.default.web_returns (18) - : : : +- * Project (29) - : : : +- * SortMergeJoin Inner (28) - : : : :- * Sort (25) - : : : : +- ReusedExchange (24) - : : : +- * Sort (27) - : : : +- ReusedExchange (26) - : : +- BroadcastExchange (37) - : : +- * ColumnarToRow (36) - : : +- CometProject (35) - : : +- CometFilter (34) - : : +- CometScan parquet spark_catalog.default.date_dim (33) - : +- BroadcastExchange (44) - : +- * ColumnarToRow (43) - : +- CometProject (42) - : +- CometFilter (41) - : +- CometScan parquet spark_catalog.default.customer_address (40) - +- BroadcastExchange (51) - +- * ColumnarToRow (50) - +- CometProject (49) - +- CometFilter (48) - +- CometScan parquet spark_catalog.default.web_site (47) +* HashAggregate (63) ++- * ColumnarToRow (62) + +- CometColumnarExchange (61) + +- RowToColumnar (60) + +- * HashAggregate (59) + +- * HashAggregate (58) + +- * HashAggregate (57) + +- * Project (56) + +- * BroadcastHashJoin Inner BuildRight (55) + :- * Project (49) + : +- * BroadcastHashJoin Inner BuildRight (48) + : :- * Project (42) + : : +- * BroadcastHashJoin Inner BuildRight (41) + : : :- * SortMergeJoin LeftSemi (35) + : : : :- * SortMergeJoin LeftSemi (18) + : : : : :- * ColumnarToRow (6) + : : : : : +- CometSort (5) + : : : : : +- CometColumnarExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : : +- * Project (17) + : : : : +- * SortMergeJoin Inner (16) + : : : : :- * ColumnarToRow (12) + : : : : : +- CometSort (11) + : : : : : +- CometColumnarExchange (10) + : : : : : +- CometProject (9) + : : : : : +- CometFilter (8) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (7) + : : : : +- * ColumnarToRow (15) + : : : : +- CometSort (14) + : : : : +- ReusedExchange (13) + : : : +- * Project (34) + : : : +- * SortMergeJoin Inner (33) + : : : :- * ColumnarToRow (24) + : : : : +- CometSort (23) + : : : : +- CometColumnarExchange (22) + : : : : +- CometProject (21) + : : : : +- CometFilter (20) + : : : : +- CometScan parquet spark_catalog.default.web_returns (19) + : : : +- * Project (32) + : : : +- * SortMergeJoin Inner (31) + : : : :- * ColumnarToRow (27) + : : : : +- CometSort (26) + : : : : +- ReusedExchange (25) + : : : +- * ColumnarToRow (30) + : : : +- CometSort (29) + : : : +- ReusedExchange (28) + : : +- BroadcastExchange (40) + : : +- * ColumnarToRow (39) + : : +- CometProject (38) + : : +- CometFilter (37) + : : +- CometScan parquet spark_catalog.default.date_dim (36) + : +- BroadcastExchange (47) + : +- * ColumnarToRow (46) + : +- CometProject (45) + : +- CometFilter (44) + : +- CometScan parquet spark_catalog.default.customer_address (43) + +- BroadcastExchange (54) + +- * ColumnarToRow (53) + +- CometProject (52) + +- CometFilter (51) + +- CometScan parquet spark_catalog.default.web_site (50) (1) Scan parquet spark_catalog.default.web_sales @@ -74,16 +79,16 @@ Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -(4) ColumnarToRow [codegen id : 1] +(4) CometColumnarExchange Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] +Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(5) Exchange +(5) CometSort Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_order_number#4 ASC NULLS FIRST] -(6) Sort [codegen id : 2] +(6) ColumnarToRow [codegen id : 1] Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Arguments: [ws_order_number#4 ASC NULLS FIRST], false, 0 (7) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] @@ -100,228 +105,243 @@ Condition : (isnotnull(ws_order_number#9) AND isnotnull(ws_warehouse_sk#8)) Input [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_warehouse_sk#8, ws_order_number#9] -(10) ColumnarToRow [codegen id : 3] +(10) CometColumnarExchange Input [2]: [ws_warehouse_sk#8, ws_order_number#9] +Arguments: hashpartitioning(ws_order_number#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(11) Exchange +(11) CometSort Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: hashpartitioning(ws_order_number#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_order_number#9 ASC NULLS FIRST] -(12) Sort [codegen id : 4] +(12) ColumnarToRow [codegen id : 2] Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: [ws_order_number#9 ASC NULLS FIRST], false, 0 -(13) ReusedExchange [Reuses operator id: 11] +(13) ReusedExchange [Reuses operator id: 10] Output [2]: [ws_warehouse_sk#11, ws_order_number#12] -(14) Sort [codegen id : 6] +(14) CometSort Input [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_order_number#12 ASC NULLS FIRST], false, 0 +Arguments: [ws_warehouse_sk#11, ws_order_number#12], [ws_order_number#12 ASC NULLS FIRST] -(15) SortMergeJoin [codegen id : 7] +(15) ColumnarToRow [codegen id : 3] +Input [2]: [ws_warehouse_sk#11, ws_order_number#12] + +(16) SortMergeJoin [codegen id : 4] Left keys [1]: [ws_order_number#9] Right keys [1]: [ws_order_number#12] Join type: Inner Join condition: NOT (ws_warehouse_sk#8 = ws_warehouse_sk#11) -(16) Project [codegen id : 7] +(17) Project [codegen id : 4] Output [1]: [ws_order_number#9] Input [4]: [ws_warehouse_sk#8, ws_order_number#9, ws_warehouse_sk#11, ws_order_number#12] -(17) SortMergeJoin [codegen id : 8] +(18) SortMergeJoin [codegen id : 5] Left keys [1]: [ws_order_number#4] Right keys [1]: [ws_order_number#9] Join type: LeftSemi Join condition: None -(18) Scan parquet spark_catalog.default.web_returns +(19) Scan parquet spark_catalog.default.web_returns Output [2]: [wr_order_number#13, wr_returned_date_sk#14] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number)] ReadSchema: struct -(19) CometFilter +(20) CometFilter Input [2]: [wr_order_number#13, wr_returned_date_sk#14] Condition : isnotnull(wr_order_number#13) -(20) CometProject +(21) CometProject Input [2]: [wr_order_number#13, wr_returned_date_sk#14] Arguments: [wr_order_number#13], [wr_order_number#13] -(21) ColumnarToRow [codegen id : 9] +(22) CometColumnarExchange Input [1]: [wr_order_number#13] +Arguments: hashpartitioning(wr_order_number#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) Exchange +(23) CometSort Input [1]: [wr_order_number#13] -Arguments: hashpartitioning(wr_order_number#13, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: [wr_order_number#13], [wr_order_number#13 ASC NULLS FIRST] -(23) Sort [codegen id : 10] +(24) ColumnarToRow [codegen id : 6] Input [1]: [wr_order_number#13] -Arguments: [wr_order_number#13 ASC NULLS FIRST], false, 0 -(24) ReusedExchange [Reuses operator id: 11] +(25) ReusedExchange [Reuses operator id: 10] Output [2]: [ws_warehouse_sk#15, ws_order_number#16] -(25) Sort [codegen id : 12] +(26) CometSort Input [2]: [ws_warehouse_sk#15, ws_order_number#16] -Arguments: [ws_order_number#16 ASC NULLS FIRST], false, 0 +Arguments: [ws_warehouse_sk#15, ws_order_number#16], [ws_order_number#16 ASC NULLS FIRST] -(26) ReusedExchange [Reuses operator id: 11] +(27) ColumnarToRow [codegen id : 7] +Input [2]: [ws_warehouse_sk#15, ws_order_number#16] + +(28) ReusedExchange [Reuses operator id: 10] Output [2]: [ws_warehouse_sk#17, ws_order_number#18] -(27) Sort [codegen id : 14] +(29) CometSort +Input [2]: [ws_warehouse_sk#17, ws_order_number#18] +Arguments: [ws_warehouse_sk#17, ws_order_number#18], [ws_order_number#18 ASC NULLS FIRST] + +(30) ColumnarToRow [codegen id : 8] Input [2]: [ws_warehouse_sk#17, ws_order_number#18] -Arguments: [ws_order_number#18 ASC NULLS FIRST], false, 0 -(28) SortMergeJoin [codegen id : 15] +(31) SortMergeJoin [codegen id : 9] Left keys [1]: [ws_order_number#16] Right keys [1]: [ws_order_number#18] Join type: Inner Join condition: NOT (ws_warehouse_sk#15 = ws_warehouse_sk#17) -(29) Project [codegen id : 15] +(32) Project [codegen id : 9] Output [1]: [ws_order_number#16] Input [4]: [ws_warehouse_sk#15, ws_order_number#16, ws_warehouse_sk#17, ws_order_number#18] -(30) SortMergeJoin [codegen id : 16] +(33) SortMergeJoin [codegen id : 10] Left keys [1]: [wr_order_number#13] Right keys [1]: [ws_order_number#16] Join type: Inner Join condition: None -(31) Project [codegen id : 16] +(34) Project [codegen id : 10] Output [1]: [wr_order_number#13] Input [2]: [wr_order_number#13, ws_order_number#16] -(32) SortMergeJoin [codegen id : 20] +(35) SortMergeJoin [codegen id : 14] Left keys [1]: [ws_order_number#4] Right keys [1]: [wr_order_number#13] Join type: LeftSemi Join condition: None -(33) Scan parquet spark_catalog.default.date_dim +(36) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#19, d_date#20] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] ReadSchema: struct -(34) CometFilter +(37) CometFilter Input [2]: [d_date_sk#19, d_date#20] Condition : (((isnotnull(d_date#20) AND (d_date#20 >= 1999-02-01)) AND (d_date#20 <= 1999-04-02)) AND isnotnull(d_date_sk#19)) -(35) CometProject +(38) CometProject Input [2]: [d_date_sk#19, d_date#20] Arguments: [d_date_sk#19], [d_date_sk#19] -(36) ColumnarToRow [codegen id : 17] +(39) ColumnarToRow [codegen id : 11] Input [1]: [d_date_sk#19] -(37) BroadcastExchange +(40) BroadcastExchange Input [1]: [d_date_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(38) BroadcastHashJoin [codegen id : 20] +(41) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ws_ship_date_sk#1] Right keys [1]: [d_date_sk#19] Join type: Inner Join condition: None -(39) Project [codegen id : 20] +(42) Project [codegen id : 14] Output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, d_date_sk#19] -(40) Scan parquet spark_catalog.default.customer_address +(43) Scan parquet spark_catalog.default.customer_address Output [2]: [ca_address_sk#21, ca_state#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,IL), IsNotNull(ca_address_sk)] ReadSchema: struct -(41) CometFilter +(44) CometFilter Input [2]: [ca_address_sk#21, ca_state#22] Condition : ((isnotnull(ca_state#22) AND (ca_state#22 = IL)) AND isnotnull(ca_address_sk#21)) -(42) CometProject +(45) CometProject Input [2]: [ca_address_sk#21, ca_state#22] Arguments: [ca_address_sk#21], [ca_address_sk#21] -(43) ColumnarToRow [codegen id : 18] +(46) ColumnarToRow [codegen id : 12] Input [1]: [ca_address_sk#21] -(44) BroadcastExchange +(47) BroadcastExchange Input [1]: [ca_address_sk#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(45) BroadcastHashJoin [codegen id : 20] +(48) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ws_ship_addr_sk#2] Right keys [1]: [ca_address_sk#21] Join type: Inner Join condition: None -(46) Project [codegen id : 20] +(49) Project [codegen id : 14] Output [4]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ca_address_sk#21] -(47) Scan parquet spark_catalog.default.web_site +(50) Scan parquet spark_catalog.default.web_site Output [2]: [web_site_sk#23, web_company_name#24] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] ReadSchema: struct -(48) CometFilter +(51) CometFilter Input [2]: [web_site_sk#23, web_company_name#24] Condition : ((isnotnull(web_company_name#24) AND (web_company_name#24 = pri )) AND isnotnull(web_site_sk#23)) -(49) CometProject +(52) CometProject Input [2]: [web_site_sk#23, web_company_name#24] Arguments: [web_site_sk#23], [web_site_sk#23] -(50) ColumnarToRow [codegen id : 19] +(53) ColumnarToRow [codegen id : 13] Input [1]: [web_site_sk#23] -(51) BroadcastExchange +(54) BroadcastExchange Input [1]: [web_site_sk#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(52) BroadcastHashJoin [codegen id : 20] +(55) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ws_web_site_sk#3] Right keys [1]: [web_site_sk#23] Join type: Inner Join condition: None -(53) Project [codegen id : 20] +(56) Project [codegen id : 14] Output [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Input [5]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, web_site_sk#23] -(54) HashAggregate [codegen id : 20] +(57) HashAggregate [codegen id : 14] Input [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Keys [1]: [ws_order_number#4] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#5)), partial_sum(UnscaledValue(ws_net_profit#6))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#25, sum(UnscaledValue(ws_net_profit#6))#26] Results [3]: [ws_order_number#4, sum#27, sum#28] -(55) HashAggregate [codegen id : 20] +(58) HashAggregate [codegen id : 14] Input [3]: [ws_order_number#4, sum#27, sum#28] Keys [1]: [ws_order_number#4] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#25, sum(UnscaledValue(ws_net_profit#6))#26] Results [3]: [ws_order_number#4, sum#27, sum#28] -(56) HashAggregate [codegen id : 20] +(59) HashAggregate [codegen id : 14] Input [3]: [ws_order_number#4, sum#27, sum#28] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6)), partial_count(distinct ws_order_number#4)] Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#25, sum(UnscaledValue(ws_net_profit#6))#26, count(ws_order_number#4)#29] Results [3]: [sum#27, sum#28, count#30] -(57) Exchange +(60) RowToColumnar +Input [3]: [sum#27, sum#28, count#30] + +(61) CometColumnarExchange +Input [3]: [sum#27, sum#28, count#30] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(62) ColumnarToRow [codegen id : 15] Input [3]: [sum#27, sum#28, count#30] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(58) HashAggregate [codegen id : 21] +(63) HashAggregate [codegen id : 15] Input [3]: [sum#27, sum#28, count#30] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#5)), sum(UnscaledValue(ws_net_profit#6)), count(distinct ws_order_number#4)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/simplified.txt index 178c25f66..2c736e6e5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/simplified.txt @@ -1,102 +1,101 @@ -WholeStageCodegen (21) +WholeStageCodegen (15) HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] - InputAdapter - Exchange #1 - WholeStageCodegen (20) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] - Project [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_date_sk,d_date_sk] - SortMergeJoin [ws_order_number,wr_order_number] - InputAdapter - WholeStageCodegen (8) - SortMergeJoin [ws_order_number,ws_order_number] - InputAdapter - WholeStageCodegen (2) - Sort [ws_order_number] - InputAdapter - Exchange [ws_order_number] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometColumnarExchange #1 + RowToColumnar + WholeStageCodegen (14) + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] + Project [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_date_sk,d_date_sk] + SortMergeJoin [ws_order_number,wr_order_number] + InputAdapter + WholeStageCodegen (5) + SortMergeJoin [ws_order_number,ws_order_number] + InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometSort [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometColumnarExchange [ws_order_number] #2 CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (7) - Project [ws_order_number] - SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] - InputAdapter - WholeStageCodegen (4) - Sort [ws_order_number] - InputAdapter - Exchange [ws_order_number] #3 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter + InputAdapter + WholeStageCodegen (4) + Project [ws_order_number] + SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + InputAdapter + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometSort [ws_warehouse_sk,ws_order_number] + CometColumnarExchange [ws_order_number] #3 CometProject [ws_warehouse_sk,ws_order_number] CometFilter [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (6) - Sort [ws_order_number] - InputAdapter - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - InputAdapter - WholeStageCodegen (16) - Project [wr_order_number] - SortMergeJoin [wr_order_number,ws_order_number] - InputAdapter - WholeStageCodegen (10) - Sort [wr_order_number] - InputAdapter - Exchange [wr_order_number] #4 - WholeStageCodegen (9) - ColumnarToRow - InputAdapter + InputAdapter + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometSort [ws_warehouse_sk,ws_order_number] + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + InputAdapter + WholeStageCodegen (10) + Project [wr_order_number] + SortMergeJoin [wr_order_number,ws_order_number] + InputAdapter + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometSort [wr_order_number] + CometColumnarExchange [wr_order_number] #4 CometProject [wr_order_number] CometFilter [wr_order_number,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] - InputAdapter - WholeStageCodegen (15) - Project [ws_order_number] - SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] - InputAdapter - WholeStageCodegen (12) - Sort [ws_order_number] - InputAdapter - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - InputAdapter - WholeStageCodegen (14) - Sort [ws_order_number] - InputAdapter - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + InputAdapter + WholeStageCodegen (9) + Project [ws_order_number] + SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + InputAdapter + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometSort [ws_warehouse_sk,ws_order_number] + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + InputAdapter + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometSort [ws_warehouse_sk,ws_order_number] + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (17) + BroadcastExchange #6 + WholeStageCodegen (12) ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (18) + BroadcastExchange #7 + WholeStageCodegen (13) ColumnarToRow InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (19) - ColumnarToRow - InputAdapter - CometProject [web_site_sk] - CometFilter [web_site_sk,web_company_name] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] + CometProject [web_site_sk] + CometFilter [web_site_sk,web_company_name] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/explain.txt index 4bc24750f..6f9d1a3f2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* HashAggregate (25) -+- Exchange (24) - +- * ColumnarToRow (23) +* ColumnarToRow (25) ++- CometHashAggregate (24) + +- CometColumnarExchange (23) +- CometHashAggregate (22) +- CometProject (21) +- CometBroadcastHashJoin (20) @@ -129,17 +129,15 @@ Input: [] Keys: [] Functions [1]: [partial_count(1)] -(23) ColumnarToRow [codegen id : 1] +(23) CometColumnarExchange Input [1]: [count#12] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(24) Exchange -Input [1]: [count#12] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] - -(25) HashAggregate [codegen id : 2] +(24) CometHashAggregate Input [1]: [count#12] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#13] -Results [1]: [count(1)#13 AS count(1)#14] + +(25) ColumnarToRow [codegen id : 1] +Input [1]: [count(1)#13] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/simplified.txt index e9d33a7f6..b8b3e211d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/simplified.txt @@ -1,29 +1,27 @@ -WholeStageCodegen (2) - HashAggregate [count] [count(1),count(1),count] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] - CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_store_name] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] + CometHashAggregate [count(1),count,count(1)] + CometColumnarExchange #1 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometBroadcastExchange [hd_demo_sk] #2 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] + CometBroadcastExchange [t_time_sk] #3 + CometProject [t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + CometBroadcastExchange [s_store_sk] #4 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_store_name] + 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-spark3_5/q97/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/explain.txt index 7508405a8..e119edec6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/explain.txt @@ -1,30 +1,29 @@ == Physical Plan == -* 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) +* ColumnarToRow (25) ++- CometHashAggregate (24) + +- CometColumnarExchange (23) + +- CometHashAggregate (22) + +- CometProject (21) + +- CometSortMergeJoin (20) + :- CometSort (11) + : +- CometHashAggregate (10) + : +- CometColumnarExchange (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) + +- CometSort (19) + +- CometHashAggregate (18) + +- CometColumnarExchange (17) + +- CometHashAggregate (16) + +- CometProject (15) + +- CometBroadcastHashJoin (14) + :- CometScan parquet spark_catalog.default.catalog_sales (12) + +- ReusedExchange (13) (1) Scan parquet spark_catalog.default.store_sales @@ -67,126 +66,114 @@ Input [2]: [ss_item_sk#1, ss_customer_sk#2] Keys [2]: [ss_customer_sk#2, ss_item_sk#1] Functions: [] -(9) ColumnarToRow [codegen id : 1] +(9) CometColumnarExchange Input [2]: [ss_customer_sk#2, ss_item_sk#1] +Arguments: hashpartitioning(ss_customer_sk#2, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=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] - -(11) HashAggregate [codegen id : 2] +(10) CometHashAggregate 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#7, ss_item_sk#1 AS item_sk#8] -(12) Sort [codegen id : 2] +(11) CometSort Input [2]: [customer_sk#7, item_sk#8] -Arguments: [customer_sk#7 ASC NULLS FIRST, item_sk#8 ASC NULLS FIRST], false, 0 +Arguments: [customer_sk#7, item_sk#8], [customer_sk#7 ASC NULLS FIRST, item_sk#8 ASC NULLS FIRST] -(13) Scan parquet spark_catalog.default.catalog_sales +(12) 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#11), dynamicpruningexpression(cs_sold_date_sk#11 IN dynamicpruning#12)] ReadSchema: struct -(14) ReusedExchange [Reuses operator id: 5] +(13) ReusedExchange [Reuses operator id: 5] Output [1]: [d_date_sk#13] -(15) CometBroadcastHashJoin +(14) 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 -(16) CometProject +(15) 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] -(17) CometHashAggregate +(16) CometHashAggregate Input [2]: [cs_bill_customer_sk#9, cs_item_sk#10] Keys [2]: [cs_bill_customer_sk#9, cs_item_sk#10] Functions: [] -(18) ColumnarToRow [codegen id : 3] +(17) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=2] -(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] - -(20) HashAggregate [codegen id : 4] +(18) 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#9 AS customer_sk#14, cs_item_sk#10 AS item_sk#15] -(21) Sort [codegen id : 4] +(19) CometSort Input [2]: [customer_sk#14, item_sk#15] -Arguments: [customer_sk#14 ASC NULLS FIRST, item_sk#15 ASC NULLS FIRST], false, 0 +Arguments: [customer_sk#14, item_sk#15], [customer_sk#14 ASC NULLS FIRST, item_sk#15 ASC NULLS FIRST] -(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) CometSortMergeJoin +Left output [2]: [customer_sk#7, item_sk#8] +Right output [2]: [customer_sk#14, item_sk#15] +Arguments: [customer_sk#7, item_sk#8], [customer_sk#14, item_sk#15], FullOuter -(23) Project [codegen id : 5] -Output [2]: [customer_sk#7, customer_sk#14] +(21) CometProject Input [4]: [customer_sk#7, item_sk#8, customer_sk#14, item_sk#15] +Arguments: [customer_sk#7, customer_sk#14], [customer_sk#7, customer_sk#14] -(24) HashAggregate [codegen id : 5] +(22) CometHashAggregate Input [2]: [customer_sk#7, customer_sk#14] Keys: [] 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] -(25) Exchange -Input [3]: [sum#19, sum#20, sum#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] +(23) CometColumnarExchange +Input [3]: [sum#16, sum#17, sum#18] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(26) HashAggregate [codegen id : 6] -Input [3]: [sum#19, sum#20, sum#21] +(24) CometHashAggregate +Input [3]: [sum#16, sum#17, sum#18] Keys: [] 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] + +(25) ColumnarToRow [codegen id : 1] +Input [3]: [store_only#19, catalog_only#20, store_and_catalog#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (31) -+- * ColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan parquet spark_catalog.default.date_dim (27) +BroadcastExchange (30) ++- * ColumnarToRow (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan parquet spark_catalog.default.date_dim (26) -(27) Scan parquet spark_catalog.default.date_dim +(26) 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 -(28) CometFilter +(27) 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)) -(29) CometProject +(28) CometProject Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(30) ColumnarToRow [codegen id : 1] +(29) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(31) BroadcastExchange +(30) 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 = 13 Hosting Expression = cs_sold_date_sk#11 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 12 Hosting Expression = cs_sold_date_sk#11 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/simplified.txt index 41f3e579f..da4461d84 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/simplified.txt @@ -1,48 +1,36 @@ -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] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange #1 - 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 (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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ss_customer_sk,ss_item_sk] - CometProject [ss_item_sk,ss_customer_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,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 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - 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] #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometHashAggregate [cs_bill_customer_sk,cs_item_sk] - CometProject [cs_bill_customer_sk,cs_item_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,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 - ReusedExchange [d_date_sk] #4 + CometHashAggregate [store_only,catalog_only,store_and_catalog,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)] + CometColumnarExchange #1 + CometHashAggregate [sum,sum,sum,customer_sk,customer_sk] + CometProject [customer_sk,customer_sk] + CometSortMergeJoin [customer_sk,item_sk,customer_sk,item_sk] + CometSort [customer_sk,item_sk] + CometHashAggregate [customer_sk,item_sk,ss_customer_sk,ss_item_sk] + CometColumnarExchange [ss_customer_sk,ss_item_sk] #2 + CometHashAggregate [ss_customer_sk,ss_item_sk] + CometProject [ss_item_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,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 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometSort [customer_sk,item_sk] + CometHashAggregate [customer_sk,item_sk,cs_bill_customer_sk,cs_item_sk] + CometColumnarExchange [cs_bill_customer_sk,cs_item_sk] #5 + CometHashAggregate [cs_bill_customer_sk,cs_item_sk] + CometProject [cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,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 + ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/explain.txt index ab7a40acd..0239cc78c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/explain.txt @@ -1,28 +1,30 @@ == Physical Plan == -* 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) +* ColumnarToRow (26) ++- CometProject (25) + +- CometSort (24) + +- CometColumnarExchange (23) + +- RowToColumnar (22) + +- * Project (21) + +- Window (20) + +- * ColumnarToRow (19) + +- CometSort (18) + +- CometColumnarExchange (17) + +- CometHashAggregate (16) + +- CometColumnarExchange (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 @@ -94,77 +96,81 @@ Input [6]: [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, 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))] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(16) CometHashAggregate 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] -(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=2] +(17) CometColumnarExchange +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(18) CometSort +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] +Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6], [i_class#9 ASC NULLS FIRST] -(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 +(19) ColumnarToRow [codegen id : 1] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] (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] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] +Arguments: [sum(_w0#15) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#16], [i_class#9] + +(21) Project [codegen id : 2] +Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, ((_w0#15 * 100) / _we0#16) AS revenueratio#17, i_item_id#6] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6, _we0#16] + +(22) RowToColumnar +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17, i_item_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] +(23) CometColumnarExchange +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17, 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#17 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(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=3] +(24) CometSort +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17, i_item_id#6] +Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17, i_item_id#6], [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#17 ASC NULLS FIRST] -(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 +(25) CometProject +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17, i_item_id#6] +Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] -(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] +(26) ColumnarToRow [codegen id : 3] +Input [6]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * ColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan parquet spark_catalog.default.date_dim (25) +BroadcastExchange (31) ++- * ColumnarToRow (30) + +- CometProject (29) + +- CometFilter (28) + +- CometScan parquet spark_catalog.default.date_dim (27) -(25) Scan parquet spark_catalog.default.date_dim +(27) 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 -(26) CometFilter +(28) 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)) -(27) CometProject +(29) CometProject Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(28) ColumnarToRow [codegen id : 1] +(30) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(29) BroadcastExchange +(31) BroadcastExchange Input [1]: [d_date_sk#11] 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-spark3_5/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/simplified.txt index 3895cdc78..663bc2e3e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/simplified.txt @@ -1,42 +1,40 @@ -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 (4) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (3) - Sort [i_class] - InputAdapter - Exchange [i_class] #2 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ss_ext_sales_price] - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_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 #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] +WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] + CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 + RowToColumnar + WholeStageCodegen (2) + Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometColumnarExchange [i_class] #2 + CometHashAggregate [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ss_ext_sales_price] + CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_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 #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + 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-spark3_5/q99/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/explain.txt index 131687112..ceb28b9e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (28) -+- * HashAggregate (27) - +- Exchange (26) - +- * ColumnarToRow (25) +* ColumnarToRow (28) ++- CometTakeOrderedAndProject (27) + +- CometHashAggregate (26) + +- CometColumnarExchange (25) +- CometHashAggregate (24) +- CometProject (23) +- CometBroadcastHashJoin (22) @@ -145,21 +145,19 @@ Input [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#9, cc_name#11, _groupi 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)] -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(27) HashAggregate [codegen id : 2] +(26) CometHashAggregate 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)#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] -(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] +(27) CometTakeOrderedAndProject +Input [8]: [substr(w_warehouse_name, 1, 20)#20, sm_type#9, cc_name#11, 30 days #21, 31 - 60 days #22, 61 - 90 days #23, 91 - 120 days #24, >120 days #25] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#20 ASC NULLS FIRST,sm_type#9 ASC NULLS FIRST,cc_name#11 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#20,sm_type#9,cc_name#11,30 days #21,31 - 60 days #22,61 - 90 days #23,91 - 120 days #24,>120 days #25]), [substr(w_warehouse_name, 1, 20)#20, sm_type#9, cc_name#11, 30 days #21, 31 - 60 days #22, 61 - 90 days #23, 91 - 120 days #24, >120 days #25], 100, [substr(w_warehouse_name, 1, 20)#20 ASC NULLS FIRST, sm_type#9 ASC NULLS FIRST, cc_name#11 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#20, sm_type#9, cc_name#11, 30 days #21, 31 - 60 days #22, 61 - 90 days #23, 91 - 120 days #24, >120 days #25] + +(28) ColumnarToRow [codegen id : 1] +Input [8]: [substr(w_warehouse_name, 1, 20)#20, sm_type#9, cc_name#11, 30 days #21, 31 - 60 days #22, 61 - 90 days #23, 91 - 120 days #24, >120 days #25] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/simplified.txt index fc63929bb..f5ca9c40a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/simplified.txt @@ -1,32 +1,30 @@ -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 (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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum,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,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name,d_date_sk] - CometProject [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name] - CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_call_center_sk,cc_name] - CometProject [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type] - CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] - CometProject [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name] - CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name] - CometFilter [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_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] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 - CometFilter [sm_ship_mode_sk,sm_type] - CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange [cc_call_center_sk,cc_name] #4 - CometFilter [cc_call_center_sk,cc_name] - CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] + CometHashAggregate [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,_groupingexpression,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)] + CometColumnarExchange [_groupingexpression,sm_type,cc_name] #1 + CometHashAggregate [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum,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,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name,d_date_sk] + CometProject [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_call_center_sk,cc_name] + CometProject [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] + CometProject [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_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] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 + CometFilter [w_warehouse_sk,w_warehouse_name] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 + CometFilter [sm_ship_mode_sk,sm_type] + CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] + CometBroadcastExchange [cc_call_center_sk,cc_name] #4 + CometFilter [cc_call_center_sk,cc_name] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + 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/q1/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/explain.txt index f3cd64a14..d1a177b29 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,47 +1,53 @@ == Physical Plan == -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) +TakeOrderedAndProject (49) ++- * Project (48) + +- * BroadcastHashJoin Inner BuildRight (47) + :- * Project (42) + : +- * BroadcastHashJoin Inner BuildRight (41) + : :- * Project (35) + : : +- * BroadcastHashJoin Inner BuildRight (34) + : : :- * Filter (15) + : : : +- * HashAggregate (14) + : : : +- * ColumnarToRow (13) + : : : +- CometColumnarExchange (12) + : : : +- RowToColumnar (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 (33) + : : +- * Filter (32) + : : +- * HashAggregate (31) + : : +- * ColumnarToRow (30) + : : +- CometColumnarExchange (29) + : : +- RowToColumnar (28) + : : +- * HashAggregate (27) + : : +- * HashAggregate (26) + : : +- * ColumnarToRow (25) + : : +- CometColumnarExchange (24) + : : +- RowToColumnar (23) + : : +- * HashAggregate (22) + : : +- * ColumnarToRow (21) + : : +- CometProject (20) + : : +- CometBroadcastHashJoin (19) + : : :- CometFilter (17) + : : : +- CometScan parquet spark_catalog.default.store_returns (16) + : : +- ReusedExchange (18) + : +- BroadcastExchange (40) + : +- * ColumnarToRow (39) + : +- CometProject (38) + : +- CometFilter (37) + : +- CometScan parquet spark_catalog.default.store (36) + +- BroadcastExchange (46) + +- * ColumnarToRow (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.customer (43) (1) Scan parquet spark_catalog.default.store_returns @@ -94,22 +100,28 @@ Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#3))] Aggregate Attributes [1]: [sum#8] Results [3]: [sr_customer_sk#1, sr_store_sk#2, sum#9] -(11) Exchange +(11) RowToColumnar 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] -(12) HashAggregate [codegen id : 7] +(12) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(13) ColumnarToRow [codegen id : 7] +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#9] + +(14) 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))#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] -(13) Filter [codegen id : 7] +(15) Filter [codegen id : 7] Input [3]: [ctr_customer_sk#11, ctr_store_sk#12, ctr_total_return#13] Condition : isnotnull(ctr_total_return#13) -(14) Scan parquet spark_catalog.default.store_returns +(16) 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 [] @@ -117,175 +129,187 @@ PartitionFilters: [isnotnull(sr_returned_date_sk#17), dynamicpruningexpression(s PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(15) CometFilter +(17) 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) -(16) ReusedExchange [Reuses operator id: 6] +(18) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#19] -(17) CometBroadcastHashJoin +(19) 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 -(18) CometProject +(20) 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] +(21) ColumnarToRow [codegen id : 2] Input [3]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16] -(20) HashAggregate [codegen id : 2] +(22) 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 +(23) RowToColumnar 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] +(24) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] + +(25) ColumnarToRow [codegen id : 3] +Input [3]: [sr_customer_sk#14, sr_store_sk#15, sum#21] + +(26) 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] +(27) 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 +(28) RowToColumnar +Input [3]: [ctr_store_sk#22, sum#26, count#27] + +(29) CometColumnarExchange +Input [3]: [ctr_store_sk#22, sum#26, count#27] +Arguments: hashpartitioning(ctr_store_sk#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(30) ColumnarToRow [codegen id : 4] 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] +(31) 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] +(32) 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 +(33) 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] -(28) BroadcastHashJoin [codegen id : 7] +(34) 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#13 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#29) -(29) Project [codegen id : 7] +(35) 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] -(30) Scan parquet spark_catalog.default.store +(36) 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 -(31) CometFilter +(37) 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)) -(32) CometProject +(38) CometProject Input [2]: [s_store_sk#30, s_state#31] Arguments: [s_store_sk#30], [s_store_sk#30] -(33) ColumnarToRow [codegen id : 5] +(39) ColumnarToRow [codegen id : 5] Input [1]: [s_store_sk#30] -(34) BroadcastExchange +(40) BroadcastExchange Input [1]: [s_store_sk#30] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(35) BroadcastHashJoin [codegen id : 7] +(41) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ctr_store_sk#12] Right keys [1]: [s_store_sk#30] Join type: Inner Join condition: None -(36) Project [codegen id : 7] +(42) Project [codegen id : 7] Output [1]: [ctr_customer_sk#11] Input [3]: [ctr_customer_sk#11, ctr_store_sk#12, s_store_sk#30] -(37) Scan parquet spark_catalog.default.customer +(43) 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 -(38) CometFilter +(44) CometFilter Input [2]: [c_customer_sk#32, c_customer_id#33] Condition : isnotnull(c_customer_sk#32) -(39) ColumnarToRow [codegen id : 6] +(45) ColumnarToRow [codegen id : 6] Input [2]: [c_customer_sk#32, c_customer_id#33] -(40) BroadcastExchange +(46) 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] -(41) BroadcastHashJoin [codegen id : 7] +(47) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ctr_customer_sk#11] Right keys [1]: [c_customer_sk#32] Join type: Inner Join condition: None -(42) Project [codegen id : 7] +(48) Project [codegen id : 7] Output [1]: [c_customer_id#33] Input [3]: [ctr_customer_sk#11, c_customer_sk#32, c_customer_id#33] -(43) TakeOrderedAndProject +(49) 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 (48) -+- * ColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan parquet spark_catalog.default.date_dim (44) +BroadcastExchange (54) ++- * ColumnarToRow (53) + +- CometProject (52) + +- CometFilter (51) + +- CometScan parquet spark_catalog.default.date_dim (50) -(44) Scan parquet spark_catalog.default.date_dim +(50) 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 -(45) CometFilter +(51) 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)) -(46) CometProject +(52) CometProject Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(47) ColumnarToRow [codegen id : 1] +(53) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(48) BroadcastExchange +(54) 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 = 14 Hosting Expression = sr_returned_date_sk#17 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 16 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 43bad5d70..b0c85f80b 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 @@ -8,50 +8,56 @@ TakeOrderedAndProject [c_customer_id] BroadcastHashJoin [ctr_store_sk,ctr_store_sk,ctr_total_return,(avg(ctr_total_return) * 1.2)] Filter [ctr_total_return] 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 (1) - HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] - CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ColumnarToRow + InputAdapter + CometColumnarExchange [sr_customer_sk,sr_store_sk] #1 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] + CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] + CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter 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] #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] #6 - WholeStageCodegen (2) - HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] - CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_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 - ReusedExchange [d_date_sk] #3 + ColumnarToRow + InputAdapter + CometColumnarExchange [ctr_store_sk] #5 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [sr_customer_sk,sr_store_sk] #6 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] + CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] + CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_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 + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #7 WholeStageCodegen (5) 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 4a29b7260..6f8c539fb 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,49 +1,51 @@ == Physical Plan == -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) +TakeOrderedAndProject (47) ++- * HashAggregate (46) + +- * ColumnarToRow (45) + +- CometColumnarExchange (44) + +- RowToColumnar (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 @@ -243,50 +245,56 @@ Functions [1]: [partial_count(1)] 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] -(43) Exchange +(43) RowToColumnar 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] -(44) HashAggregate [codegen id : 6] +(44) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=5] + +(45) ColumnarToRow [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] + +(46) 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)#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] -(45) TakeOrderedAndProject +(47) 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 = 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) +BroadcastExchange (52) ++- * ColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) -(46) Scan parquet spark_catalog.default.date_dim +(48) 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 -(47) CometFilter +(49) 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)) -(48) CometProject +(50) CometProject Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(49) ColumnarToRow [codegen id : 1] +(51) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(50) BroadcastExchange +(52) BroadcastExchange Input [1]: [d_date_sk#9] 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/q10/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/simplified.txt index efd4b187d..a3ab6bbbb 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,71 +1,73 @@ 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 (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 (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] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] - CometFilter [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] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,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 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometColumnarExchange [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 + RowToColumnar + 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] + Project [c_current_cdemo_sk] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] + Filter [exists,exists] + BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,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 + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [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] + CometBroadcastExchange [ss_customer_sk] #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,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 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [cs_ship_customer_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,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 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_county] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) + BroadcastExchange #8 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [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] - 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] + CometFilter [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] + 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-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 3f5dd24f2..2707bf8e3 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 @@ -1,76 +1,84 @@ == Physical Plan == -TakeOrderedAndProject (72) -+- * Project (71) - +- * BroadcastHashJoin Inner BuildRight (70) - :- * 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) - +- * 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) +TakeOrderedAndProject (80) ++- * Project (79) + +- * BroadcastHashJoin Inner BuildRight (78) + :- * Project (60) + : +- * BroadcastHashJoin Inner BuildRight (59) + : :- * Project (40) + : : +- * BroadcastHashJoin Inner BuildRight (39) + : : :- * Filter (19) + : : : +- * HashAggregate (18) + : : : +- * ColumnarToRow (17) + : : : +- CometColumnarExchange (16) + : : : +- RowToColumnar (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 (38) + : : +- * HashAggregate (37) + : : +- * ColumnarToRow (36) + : : +- CometColumnarExchange (35) + : : +- RowToColumnar (34) + : : +- * HashAggregate (33) + : : +- * ColumnarToRow (32) + : : +- CometProject (31) + : : +- CometBroadcastHashJoin (30) + : : :- CometProject (26) + : : : +- CometBroadcastHashJoin (25) + : : : :- CometFilter (21) + : : : : +- CometScan parquet spark_catalog.default.customer (20) + : : : +- CometBroadcastExchange (24) + : : : +- CometFilter (23) + : : : +- CometScan parquet spark_catalog.default.store_sales (22) + : : +- CometBroadcastExchange (29) + : : +- CometFilter (28) + : : +- CometScan parquet spark_catalog.default.date_dim (27) + : +- BroadcastExchange (58) + : +- * Filter (57) + : +- * HashAggregate (56) + : +- * ColumnarToRow (55) + : +- CometColumnarExchange (54) + : +- RowToColumnar (53) + : +- * HashAggregate (52) + : +- * ColumnarToRow (51) + : +- CometProject (50) + : +- CometBroadcastHashJoin (49) + : :- CometProject (47) + : : +- CometBroadcastHashJoin (46) + : : :- CometFilter (42) + : : : +- CometScan parquet spark_catalog.default.customer (41) + : : +- CometBroadcastExchange (45) + : : +- CometFilter (44) + : : +- CometScan parquet spark_catalog.default.web_sales (43) + : +- ReusedExchange (48) + +- BroadcastExchange (77) + +- * HashAggregate (76) + +- * ColumnarToRow (75) + +- CometColumnarExchange (74) + +- RowToColumnar (73) + +- * HashAggregate (72) + +- * ColumnarToRow (71) + +- CometProject (70) + +- CometBroadcastHashJoin (69) + :- CometProject (67) + : +- CometBroadcastHashJoin (66) + : :- CometFilter (62) + : : +- CometScan parquet spark_catalog.default.customer (61) + : +- CometBroadcastExchange (65) + : +- CometFilter (64) + : +- CometScan parquet spark_catalog.default.web_sales (63) + +- ReusedExchange (68) (1) Scan parquet spark_catalog.default.customer @@ -143,33 +151,39 @@ Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discoun 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] -(15) Exchange +(15) RowToColumnar 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=1] -(16) HashAggregate [codegen id : 8] +(16) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(17) ColumnarToRow [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] + +(18) 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] -(17) Filter [codegen id : 8] +(19) Filter [codegen id : 8] Input [2]: [customer_id#19, year_total#20] Condition : (isnotnull(year_total#20) AND (year_total#20 > 0.00)) -(18) Scan parquet spark_catalog.default.customer +(20) 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 -(19) CometFilter +(21) 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)) -(20) Scan parquet spark_catalog.default.store_sales +(22) 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 Location: InMemoryFileIndex [] @@ -177,94 +191,100 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#32), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(21) CometFilter +(23) 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) -(22) CometBroadcastExchange +(24) CometBroadcastExchange Input [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] Arguments: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] -(23) CometBroadcastHashJoin +(25) CometBroadcastHashJoin Left 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] Right output [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] Arguments: [c_customer_sk#21], [ss_customer_sk#29], Inner, BuildRight -(24) CometProject +(26) CometProject 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] 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, ss_sold_date_sk#32], [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] -(25) Scan parquet spark_catalog.default.date_dim +(27) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#34, d_year#35] 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 -(26) CometFilter +(28) 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 +(29) CometBroadcastExchange Input [2]: [d_date_sk#34, d_year#35] Arguments: [d_date_sk#34, d_year#35] -(28) CometBroadcastHashJoin +(30) 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 +(31) 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] +(32) 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] -(31) HashAggregate [codegen id : 2] +(33) 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] -(32) Exchange +(34) RowToColumnar +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] + +(35) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] + +(36) ColumnarToRow [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] -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] -(33) HashAggregate [codegen id : 3] +(37) 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] -(34) BroadcastExchange +(38) BroadcastExchange Input [3]: [customer_id#38, customer_preferred_cust_flag#39, year_total#40] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] -(35) BroadcastHashJoin [codegen id : 8] +(39) BroadcastHashJoin [codegen id : 8] Left keys [1]: [customer_id#19] Right keys [1]: [customer_id#38] Join type: Inner Join condition: None -(36) Project [codegen id : 8] +(40) 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] -(37) Scan parquet spark_catalog.default.customer +(41) 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 -(38) CometFilter +(42) 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)) -(39) Scan parquet spark_catalog.default.web_sales +(43) 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 [] @@ -272,86 +292,92 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#52), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(40) CometFilter +(44) 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) -(41) CometBroadcastExchange +(45) 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] -(42) CometBroadcastHashJoin +(46) 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) CometProject +(47) 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: 10] +(48) ReusedExchange [Reuses operator id: 10] Output [2]: [d_date_sk#54, d_year#55] -(45) CometBroadcastHashJoin +(49) 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) CometProject +(50) 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) ColumnarToRow [codegen id : 4] +(51) 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] +(52) 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] -(49) Exchange +(53) RowToColumnar +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] + +(54) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=4] + +(55) ColumnarToRow [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] -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] -(50) HashAggregate [codegen id : 5] +(56) 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] -(51) Filter [codegen id : 5] +(57) Filter [codegen id : 5] Input [2]: [customer_id#59, year_total#60] Condition : (isnotnull(year_total#60) AND (year_total#60 > 0.00)) -(52) BroadcastExchange +(58) BroadcastExchange Input [2]: [customer_id#59, year_total#60] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(53) BroadcastHashJoin [codegen id : 8] +(59) BroadcastHashJoin [codegen id : 8] Left keys [1]: [customer_id#19] Right keys [1]: [customer_id#59] Join type: Inner Join condition: None -(54) Project [codegen id : 8] +(60) 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] -(55) Scan parquet spark_catalog.default.customer +(61) 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 -(56) CometFilter +(62) 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)) -(57) Scan parquet spark_catalog.default.web_sales +(63) 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 Location: InMemoryFileIndex [] @@ -359,128 +385,134 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#72), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(58) CometFilter +(64) 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) -(59) CometBroadcastExchange +(65) CometBroadcastExchange Input [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] Arguments: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] -(60) CometBroadcastHashJoin +(66) CometBroadcastHashJoin Left 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] Right output [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] Arguments: [c_customer_sk#61], [ws_bill_customer_sk#69], Inner, BuildRight -(61) CometProject +(67) CometProject 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] Arguments: [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], [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] -(62) ReusedExchange [Reuses operator id: 27] +(68) ReusedExchange [Reuses operator id: 29] Output [2]: [d_date_sk#74, d_year#75] -(63) CometBroadcastHashJoin +(69) CometBroadcastHashJoin Left 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] Right output [2]: [d_date_sk#74, d_year#75] Arguments: [ws_sold_date_sk#72], [d_date_sk#74], Inner, BuildRight -(64) CometProject +(70) CometProject 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] Arguments: [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], [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] -(65) ColumnarToRow [codegen id : 6] +(71) ColumnarToRow [codegen id : 6] 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] -(66) HashAggregate [codegen id : 6] +(72) HashAggregate [codegen id : 6] 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] -(67) Exchange +(73) RowToColumnar +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] + +(74) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=6] + +(75) ColumnarToRow [codegen id : 7] 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=6] -(68) HashAggregate [codegen id : 7] +(76) HashAggregate [codegen id : 7] 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] -(69) BroadcastExchange +(77) BroadcastExchange Input [2]: [customer_id#78, year_total#79] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] -(70) BroadcastHashJoin [codegen id : 8] +(78) BroadcastHashJoin [codegen id : 8] Left keys [1]: [customer_id#19] Right keys [1]: [customer_id#78] 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) -(71) Project [codegen id : 8] +(79) 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] -(72) TakeOrderedAndProject +(80) TakeOrderedAndProject Input [1]: [customer_preferred_cust_flag#39] Arguments: 100, [customer_preferred_cust_flag#39 ASC NULLS FIRST], [customer_preferred_cust_flag#39] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (76) -+- * ColumnarToRow (75) - +- CometFilter (74) - +- CometScan parquet spark_catalog.default.date_dim (73) +BroadcastExchange (84) ++- * ColumnarToRow (83) + +- CometFilter (82) + +- CometScan parquet spark_catalog.default.date_dim (81) -(73) Scan parquet spark_catalog.default.date_dim +(81) 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 -(74) CometFilter +(82) 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)) -(75) ColumnarToRow [codegen id : 1] +(83) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#14, d_year#15] -(76) BroadcastExchange +(84) BroadcastExchange Input [2]: [d_date_sk#14, d_year#15] 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) -+- * ColumnarToRow (79) - +- CometFilter (78) - +- CometScan parquet spark_catalog.default.date_dim (77) +Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#33 +BroadcastExchange (88) ++- * ColumnarToRow (87) + +- CometFilter (86) + +- CometScan parquet spark_catalog.default.date_dim (85) -(77) Scan parquet spark_catalog.default.date_dim +(85) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#34, d_year#35] 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 +(86) 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)) -(79) ColumnarToRow [codegen id : 1] +(87) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#34, d_year#35] -(80) BroadcastExchange +(88) BroadcastExchange Input [2]: [d_date_sk#34, d_year#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#13 +Subquery:3 Hosting operator id = 43 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 +Subquery:4 Hosting operator id = 63 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 240d88bb7..456639607 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 @@ -8,38 +8,11 @@ TakeOrderedAndProject [customer_preferred_cust_flag] BroadcastHashJoin [customer_id,customer_id] Filter [year_total] 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 (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] - 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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_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_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - 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 + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 + RowToColumnar + 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] ColumnarToRow InputAdapter @@ -49,59 +22,94 @@ TakeOrderedAndProject [customer_preferred_cust_flag] CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] 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 [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_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 #8 + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 + CometBroadcastExchange [d_date_sk,d_year] #4 CometFilter [d_date_sk,d_year] 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #6 + RowToColumnar + 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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + 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 [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_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 #8 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter 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] #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] - 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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 - CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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 + ColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 + RowToColumnar + 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] + 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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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 #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] #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] - 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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 - CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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 + ColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 + RowToColumnar + 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] + 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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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 d7d56bf6b..a97ace27a 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,26 +1,30 @@ == Physical Plan == -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) +TakeOrderedAndProject (26) ++- * Project (25) + +- Window (24) + +- * ColumnarToRow (23) + +- CometSort (22) + +- CometColumnarExchange (21) + +- RowToColumnar (20) + +- * HashAggregate (19) + +- * ColumnarToRow (18) + +- CometColumnarExchange (17) + +- RowToColumnar (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 @@ -97,66 +101,78 @@ Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] 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] -(16) Exchange +(16) RowToColumnar 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] -(17) HashAggregate [codegen id : 2] +(17) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(18) ColumnarToRow [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] + +(19) 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))#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] -(18) Exchange +(20) RowToColumnar +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17, i_item_id#6] + +(21) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] + +(22) CometSort 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] +Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17, i_item_id#6], [i_class#9 ASC NULLS FIRST] -(19) Sort [codegen id : 3] +(23) ColumnarToRow [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 -(20) Window +(24) 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] -(21) Project [codegen id : 4] +(25) 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] -(22) TakeOrderedAndProject +(26) 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 (27) -+- * ColumnarToRow (26) - +- CometProject (25) - +- CometFilter (24) - +- CometScan parquet spark_catalog.default.date_dim (23) +BroadcastExchange (31) ++- * ColumnarToRow (30) + +- CometProject (29) + +- CometFilter (28) + +- CometScan parquet spark_catalog.default.date_dim (27) -(23) Scan parquet spark_catalog.default.date_dim +(27) 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 +(28) 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 +(29) CometProject Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(26) ColumnarToRow [codegen id : 1] +(30) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(27) BroadcastExchange +(31) BroadcastExchange Input [1]: [d_date_sk#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/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt index 1640fa3b0..9ca64cd0e 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 @@ -4,35 +4,39 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter Window [_w0,i_class] WholeStageCodegen (3) - Sort [i_class] + ColumnarToRow InputAdapter - Exchange [i_class] #1 - 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 (1) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometColumnarExchange [i_class] #1 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + 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 e137f931f..7cb01ba03 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,37 +1,39 @@ == Physical Plan == -* 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) +* HashAggregate (35) ++- * ColumnarToRow (34) + +- CometColumnarExchange (33) + +- RowToColumnar (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 @@ -184,11 +186,17 @@ Functions [4]: [partial_avg(ss_quantity#5), partial_avg(UnscaledValue(ss_ext_sal 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] -(32) Exchange +(32) RowToColumnar Input [7]: [sum#30, count#31, sum#32, count#33, sum#34, count#35, sum#36] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] -(33) HashAggregate [codegen id : 2] +(33) CometColumnarExchange +Input [7]: [sum#30, count#31, sum#32, count#33, sum#34, count#35, sum#36] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(34) ColumnarToRow [codegen id : 2] +Input [7]: [sum#30, count#31, sum#32, count#33, sum#34, count#35, sum#36] + +(35) 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))] @@ -198,32 +206,32 @@ Results [4]: [avg(ss_quantity#5)#37 AS avg(ss_quantity)#41, cast((avg(UnscaledVa ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (38) -+- * ColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.date_dim (34) +BroadcastExchange (40) ++- * ColumnarToRow (39) + +- CometProject (38) + +- CometFilter (37) + +- CometScan parquet spark_catalog.default.date_dim (36) -(34) Scan parquet spark_catalog.default.date_dim +(36) 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 -(35) CometFilter +(37) 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)) -(36) CometProject +(38) CometProject Input [2]: [d_date_sk#16, d_year#17] Arguments: [d_date_sk#16], [d_date_sk#16] -(37) ColumnarToRow [codegen id : 1] +(39) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#16] -(38) BroadcastExchange +(40) BroadcastExchange Input [1]: [d_date_sk#16] 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 cc52bb323..bbabebeea 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,45 +1,47 @@ 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 (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] - ColumnarToRow - InputAdapter - CometProject [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] - CometBroadcastHashJoin [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status,hd_demo_sk,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,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_demo_sk,cd_marital_status,cd_education_status] - CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost] - CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,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_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,ca_address_sk,ca_state] - 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_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,s_store_sk] - CometFilter [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] - 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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk] #3 - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange [ca_address_sk,ca_state] #4 - CometProject [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #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 [hd_demo_sk,hd_dep_count] #7 - CometFilter [hd_demo_sk,hd_dep_count] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] + ColumnarToRow + InputAdapter + CometColumnarExchange #1 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometProject [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] + CometBroadcastHashJoin [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status,hd_demo_sk,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,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_demo_sk,cd_marital_status,cd_education_status] + CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,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_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,ca_address_sk,ca_state] + 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_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,s_store_sk] + CometFilter [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] + 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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [s_store_sk] #3 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] + CometBroadcastExchange [ca_address_sk,ca_state] #4 + CometProject [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #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 [hd_demo_sk,hd_dep_count] #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 dc25c6e68..93d1685a5 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 @@ -1,109 +1,116 @@ == Physical Plan == -TakeOrderedAndProject (105) -+- * HashAggregate (104) - +- Exchange (103) - +- * HashAggregate (102) - +- * Expand (101) - +- Union (100) - :- * Project (67) - : +- * Filter (66) - : +- * HashAggregate (65) - : +- Exchange (64) - : +- * HashAggregate (63) - : +- * Project (62) - : +- * BroadcastHashJoin Inner BuildRight (61) - : :- * Project (59) - : : +- * BroadcastHashJoin Inner BuildRight (58) - : : :- * BroadcastHashJoin LeftSemi BuildRight (51) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (50) - : : : +- * Project (49) - : : : +- * BroadcastHashJoin Inner BuildRight (48) - : : : :- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.item (4) - : : : +- BroadcastExchange (47) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) - : : : :- * HashAggregate (35) - : : : : +- Exchange (34) - : : : : +- * 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) - : : : +- * 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) - : : : +- CometFilter (53) - : : : +- CometScan parquet spark_catalog.default.item (52) - : : +- ReusedExchange (55) - : +- ReusedExchange (60) - :- * Project (83) - : +- * Filter (82) - : +- * HashAggregate (81) - : +- Exchange (80) - : +- * HashAggregate (79) - : +- * Project (78) - : +- * BroadcastHashJoin Inner BuildRight (77) - : :- * Project (75) - : : +- * BroadcastHashJoin Inner BuildRight (74) - : : :- * BroadcastHashJoin LeftSemi BuildRight (72) - : : : :- * ColumnarToRow (70) - : : : : +- CometFilter (69) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (68) - : : : +- ReusedExchange (71) - : : +- ReusedExchange (73) - : +- ReusedExchange (76) - +- * Project (99) - +- * Filter (98) - +- * HashAggregate (97) - +- Exchange (96) - +- * HashAggregate (95) - +- * Project (94) - +- * BroadcastHashJoin Inner BuildRight (93) - :- * Project (91) - : +- * BroadcastHashJoin Inner BuildRight (90) - : :- * BroadcastHashJoin LeftSemi BuildRight (88) - : : :- * ColumnarToRow (86) - : : : +- CometFilter (85) - : : : +- CometScan parquet spark_catalog.default.web_sales (84) - : : +- ReusedExchange (87) - : +- ReusedExchange (89) - +- ReusedExchange (92) +TakeOrderedAndProject (112) ++- * HashAggregate (111) + +- * ColumnarToRow (110) + +- CometColumnarExchange (109) + +- RowToColumnar (108) + +- * HashAggregate (107) + +- * Expand (106) + +- Union (105) + :- * Project (68) + : +- * Filter (67) + : +- * HashAggregate (66) + : +- * ColumnarToRow (65) + : +- CometColumnarExchange (64) + : +- RowToColumnar (63) + : +- * HashAggregate (62) + : +- * ColumnarToRow (61) + : +- CometProject (60) + : +- CometBroadcastHashJoin (59) + : :- CometProject (54) + : : +- CometBroadcastHashJoin (53) + : : :- CometBroadcastHashJoin (47) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (46) + : : : +- CometProject (45) + : : : +- CometBroadcastHashJoin (44) + : : : :- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.item (3) + : : : +- CometBroadcastExchange (43) + : : : +- CometBroadcastHashJoin (42) + : : : :- CometHashAggregate (32) + : : : : +- CometColumnarExchange (31) + : : : : +- CometHashAggregate (30) + : : : : +- CometProject (29) + : : : : +- CometBroadcastHashJoin (28) + : : : : :- CometProject (26) + : : : : : +- CometBroadcastHashJoin (25) + : : : : : :- CometFilter (6) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (5) + : : : : : +- CometBroadcastExchange (24) + : : : : : +- CometBroadcastHashJoin (23) + : : : : : :- CometFilter (8) + : : : : : : +- CometScan parquet spark_catalog.default.item (7) + : : : : : +- CometBroadcastExchange (22) + : : : : : +- CometProject (21) + : : : : : +- CometBroadcastHashJoin (20) + : : : : : :- CometProject (15) + : : : : : : +- CometBroadcastHashJoin (14) + : : : : : : :- CometFilter (10) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (9) + : : : : : : +- CometBroadcastExchange (13) + : : : : : : +- CometFilter (12) + : : : : : : +- CometScan parquet spark_catalog.default.item (11) + : : : : : +- CometBroadcastExchange (19) + : : : : : +- CometProject (18) + : : : : : +- CometFilter (17) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (16) + : : : : +- ReusedExchange (27) + : : : +- CometBroadcastExchange (41) + : : : +- CometProject (40) + : : : +- CometBroadcastHashJoin (39) + : : : :- CometProject (37) + : : : : +- CometBroadcastHashJoin (36) + : : : : :- CometFilter (34) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (33) + : : : : +- ReusedExchange (35) + : : : +- ReusedExchange (38) + : : +- CometBroadcastExchange (52) + : : +- CometBroadcastHashJoin (51) + : : :- CometFilter (49) + : : : +- CometScan parquet spark_catalog.default.item (48) + : : +- ReusedExchange (50) + : +- CometBroadcastExchange (58) + : +- CometProject (57) + : +- CometFilter (56) + : +- CometScan parquet spark_catalog.default.date_dim (55) + :- * Project (86) + : +- * Filter (85) + : +- * HashAggregate (84) + : +- * ColumnarToRow (83) + : +- CometColumnarExchange (82) + : +- RowToColumnar (81) + : +- * HashAggregate (80) + : +- * ColumnarToRow (79) + : +- CometProject (78) + : +- CometBroadcastHashJoin (77) + : :- CometProject (75) + : : +- CometBroadcastHashJoin (74) + : : :- CometBroadcastHashJoin (72) + : : : :- CometFilter (70) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (69) + : : : +- ReusedExchange (71) + : : +- ReusedExchange (73) + : +- ReusedExchange (76) + +- * Project (104) + +- * Filter (103) + +- * HashAggregate (102) + +- * ColumnarToRow (101) + +- CometColumnarExchange (100) + +- RowToColumnar (99) + +- * HashAggregate (98) + +- * ColumnarToRow (97) + +- CometProject (96) + +- CometBroadcastHashJoin (95) + :- CometProject (93) + : +- CometBroadcastHashJoin (92) + : :- CometBroadcastHashJoin (90) + : : :- CometFilter (88) + : : : +- CometScan parquet spark_catalog.default.web_sales (87) + : : +- ReusedExchange (89) + : +- ReusedExchange (91) + +- ReusedExchange (94) (1) Scan parquet spark_catalog.default.store_sales @@ -118,24 +125,18 @@ 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 : 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 +(3) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) -(6) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(7) Scan parquet spark_catalog.default.store_sales +(5) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] @@ -143,22 +144,22 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(8) CometFilter +(6) CometFilter Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(9) Scan parquet spark_catalog.default.item +(7) 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 -(10) CometFilter +(8) 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)) -(11) Scan parquet spark_catalog.default.catalog_sales +(9) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] @@ -166,116 +167,111 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(12) CometFilter +(10) CometFilter Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) -(13) Scan parquet spark_catalog.default.item +(11) 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 -(14) CometFilter +(12) 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) -(15) CometBroadcastExchange +(13) 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] -(16) CometBroadcastHashJoin +(14) 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 -(17) CometProject +(15) 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 +(16) 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 -(19) CometFilter +(17) 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 +(18) CometProject Input [2]: [d_date_sk#24, d_year#25] Arguments: [d_date_sk#24], [d_date_sk#24] -(21) CometBroadcastExchange +(19) CometBroadcastExchange Input [1]: [d_date_sk#24] Arguments: [d_date_sk#24] -(22) CometBroadcastHashJoin +(20) 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 +(21) 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] -(24) CometBroadcastExchange +(22) CometBroadcastExchange Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] -(25) CometBroadcastHashJoin +(23) 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 -(26) CometBroadcastExchange +(24) CometBroadcastExchange Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(27) CometBroadcastHashJoin +(25) 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 -(28) CometProject +(26) 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] -(29) ReusedExchange [Reuses operator id: 21] +(27) ReusedExchange [Reuses operator id: 19] Output [1]: [d_date_sk#26] -(30) CometBroadcastHashJoin +(28) 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 -(31) CometProject +(29) 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] -(32) CometHashAggregate +(30) CometHashAggregate Input [3]: [brand_id#27, class_id#28, category_id#29] Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] -(33) ColumnarToRow [codegen id : 1] -Input [3]: [brand_id#27, class_id#28, category_id#29] - -(34) Exchange +(31) CometColumnarExchange 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] +Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(35) HashAggregate [codegen id : 3] +(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#27, class_id#28, category_id#29] -(36) Scan parquet spark_catalog.default.web_sales +(33) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] @@ -283,508 +279,541 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(37) CometFilter +(34) CometFilter Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] Condition : isnotnull(ws_item_sk#30) -(38) ReusedExchange [Reuses operator id: 15] +(35) ReusedExchange [Reuses operator id: 13] Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -(39) CometBroadcastHashJoin +(36) 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) CometProject +(37) 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) ReusedExchange [Reuses operator id: 21] +(38) ReusedExchange [Reuses operator id: 19] Output [1]: [d_date_sk#37] -(42) CometBroadcastHashJoin +(39) 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) CometProject +(40) 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) ColumnarToRow [codegen id : 2] -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(45) BroadcastExchange +(41) CometBroadcastExchange 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] +Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] -(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 +(42) CometBroadcastHashJoin +Left output [3]: [brand_id#27, class_id#28, category_id#29] +Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [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)], [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)], LeftSemi, BuildRight -(47) BroadcastExchange +(43) CometBroadcastExchange 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] +Arguments: [brand_id#27, class_id#28, category_id#29] -(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#27, class_id#28, category_id#29] -Join type: Inner -Join condition: None +(44) CometBroadcastHashJoin +Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Right output [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight -(49) Project [codegen id : 4] -Output [1]: [i_item_sk#6 AS ss_item_sk#38] +(45) CometProject 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] +Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] -(50) BroadcastExchange +(46) CometBroadcastExchange Input [1]: [ss_item_sk#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: [ss_item_sk#38] -(51) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#38] -Join type: LeftSemi -Join condition: None +(47) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Right output [1]: [ss_item_sk#38] +Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight -(52) Scan parquet spark_catalog.default.item +(48) Scan parquet spark_catalog.default.item 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 +(49) CometFilter 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 : 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] +(50) ReusedExchange [Reuses operator id: 46] Output [1]: [ss_item_sk#38] -(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 +(51) CometBroadcastHashJoin +Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Right output [1]: [ss_item_sk#38] +Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight -(57) BroadcastExchange +(52) CometBroadcastExchange 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] +Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -(58) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#39] -Join type: Inner -Join condition: None +(53) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight -(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] +(54) CometProject 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] +Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -(60) ReusedExchange [Reuses operator id: 127] -Output [1]: [d_date_sk#43] +(55) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#43, d_year#44, d_moy#45] +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 -(61) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#43] -Join type: Inner -Join condition: None +(56) CometFilter +Input [3]: [d_date_sk#43, 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 = 11)) AND isnotnull(d_date_sk#43)) -(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] +(57) CometProject +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Arguments: [d_date_sk#43], [d_date_sk#43] + +(58) CometBroadcastExchange +Input [1]: [d_date_sk#43] +Arguments: [d_date_sk#43] + +(59) CometBroadcastHashJoin +Left 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] +Right output [1]: [d_date_sk#43] +Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight + +(60) CometProject 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] +Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -(63) HashAggregate [codegen id : 11] +(61) ColumnarToRow [codegen id : 1] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] + +(62) HashAggregate [codegen id : 1] 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#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] +Aggregate Attributes [3]: [sum#46, isEmpty#47, count#48] +Results [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#49, isEmpty#50, count#51] + +(63) RowToColumnar +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#49, isEmpty#50, count#51] -(64) Exchange -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] +(64) CometColumnarExchange +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#49, isEmpty#50, count#51] +Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(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] +(65) ColumnarToRow [codegen id : 2] +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#49, isEmpty#50, count#51] + +(66) HashAggregate [codegen id : 2] +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#49, isEmpty#50, count#51] 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))#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] +Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#52, count(1)#53] +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))#52 AS sales#54, count(1)#53 AS number_sales#55] -(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) Filter [codegen id : 2] +Input [5]: [i_brand_id#40, i_class_id#41, i_category_id#42, sales#54, number_sales#55] +Condition : (isnotnull(sales#54) AND (cast(sales#54 as decimal(32,6)) > cast(Subquery scalar-subquery#56, [id=#57] as decimal(32,6)))) -(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) Project [codegen id : 2] +Output [6]: [sales#54, number_sales#55, store AS channel#58, i_brand_id#40 AS i_brand_id#59, i_class_id#41 AS i_class_id#60, i_category_id#42 AS i_category_id#61] +Input [5]: [i_brand_id#40, i_class_id#41, i_category_id#42, sales#54, number_sales#55] -(68) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#60, cs_quantity#61, cs_list_price#62, cs_sold_date_sk#63] +(69) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_item_sk#62, cs_quantity#63, cs_list_price#64, cs_sold_date_sk#65] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#63), dynamicpruningexpression(cs_sold_date_sk#63 IN dynamicpruning#64)] +PartitionFilters: [isnotnull(cs_sold_date_sk#65), dynamicpruningexpression(cs_sold_date_sk#65 IN dynamicpruning#66)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(69) CometFilter -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 : 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#65] - -(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#66, i_brand_id#67, i_class_id#68, i_category_id#69] - -(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 : 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: 127] -Output [1]: [d_date_sk#70] - -(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 : 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 : 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#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 : 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 : 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 : 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#82, ws_quantity#83, ws_list_price#84, ws_sold_date_sk#85] +(70) CometFilter +Input [4]: [cs_item_sk#62, cs_quantity#63, cs_list_price#64, cs_sold_date_sk#65] +Condition : isnotnull(cs_item_sk#62) + +(71) ReusedExchange [Reuses operator id: 46] +Output [1]: [ss_item_sk#67] + +(72) CometBroadcastHashJoin +Left output [4]: [cs_item_sk#62, cs_quantity#63, cs_list_price#64, cs_sold_date_sk#65] +Right output [1]: [ss_item_sk#67] +Arguments: [cs_item_sk#62], [ss_item_sk#67], LeftSemi, BuildRight + +(73) ReusedExchange [Reuses operator id: 52] +Output [4]: [i_item_sk#68, i_brand_id#69, i_class_id#70, i_category_id#71] + +(74) CometBroadcastHashJoin +Left output [4]: [cs_item_sk#62, cs_quantity#63, cs_list_price#64, cs_sold_date_sk#65] +Right output [4]: [i_item_sk#68, i_brand_id#69, i_class_id#70, i_category_id#71] +Arguments: [cs_item_sk#62], [i_item_sk#68], Inner, BuildRight + +(75) CometProject +Input [8]: [cs_item_sk#62, cs_quantity#63, cs_list_price#64, cs_sold_date_sk#65, i_item_sk#68, i_brand_id#69, i_class_id#70, i_category_id#71] +Arguments: [cs_quantity#63, cs_list_price#64, cs_sold_date_sk#65, i_brand_id#69, i_class_id#70, i_category_id#71], [cs_quantity#63, cs_list_price#64, cs_sold_date_sk#65, i_brand_id#69, i_class_id#70, i_category_id#71] + +(76) ReusedExchange [Reuses operator id: 58] +Output [1]: [d_date_sk#72] + +(77) CometBroadcastHashJoin +Left output [6]: [cs_quantity#63, cs_list_price#64, cs_sold_date_sk#65, i_brand_id#69, i_class_id#70, i_category_id#71] +Right output [1]: [d_date_sk#72] +Arguments: [cs_sold_date_sk#65], [d_date_sk#72], Inner, BuildRight + +(78) CometProject +Input [7]: [cs_quantity#63, cs_list_price#64, cs_sold_date_sk#65, i_brand_id#69, i_class_id#70, i_category_id#71, d_date_sk#72] +Arguments: [cs_quantity#63, cs_list_price#64, i_brand_id#69, i_class_id#70, i_category_id#71], [cs_quantity#63, cs_list_price#64, i_brand_id#69, i_class_id#70, i_category_id#71] + +(79) ColumnarToRow [codegen id : 3] +Input [5]: [cs_quantity#63, cs_list_price#64, i_brand_id#69, i_class_id#70, i_category_id#71] + +(80) HashAggregate [codegen id : 3] +Input [5]: [cs_quantity#63, cs_list_price#64, i_brand_id#69, i_class_id#70, i_category_id#71] +Keys [3]: [i_brand_id#69, i_class_id#70, i_category_id#71] +Functions [2]: [partial_sum((cast(cs_quantity#63 as decimal(10,0)) * cs_list_price#64)), partial_count(1)] +Aggregate Attributes [3]: [sum#73, isEmpty#74, count#75] +Results [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#76, isEmpty#77, count#78] + +(81) RowToColumnar +Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#76, isEmpty#77, count#78] + +(82) CometColumnarExchange +Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#76, isEmpty#77, count#78] +Arguments: hashpartitioning(i_brand_id#69, i_class_id#70, i_category_id#71, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(83) ColumnarToRow [codegen id : 4] +Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#76, isEmpty#77, count#78] + +(84) HashAggregate [codegen id : 4] +Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#76, isEmpty#77, count#78] +Keys [3]: [i_brand_id#69, i_class_id#70, i_category_id#71] +Functions [2]: [sum((cast(cs_quantity#63 as decimal(10,0)) * cs_list_price#64)), count(1)] +Aggregate Attributes [2]: [sum((cast(cs_quantity#63 as decimal(10,0)) * cs_list_price#64))#79, count(1)#80] +Results [5]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum((cast(cs_quantity#63 as decimal(10,0)) * cs_list_price#64))#79 AS sales#81, count(1)#80 AS number_sales#82] + +(85) Filter [codegen id : 4] +Input [5]: [i_brand_id#69, i_class_id#70, i_category_id#71, sales#81, number_sales#82] +Condition : (isnotnull(sales#81) AND (cast(sales#81 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#56, [id=#57] as decimal(32,6)))) + +(86) Project [codegen id : 4] +Output [6]: [sales#81, number_sales#82, catalog AS channel#83, i_brand_id#69, i_class_id#70, i_category_id#71] +Input [5]: [i_brand_id#69, i_class_id#70, i_category_id#71, sales#81, number_sales#82] + +(87) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#84, ws_quantity#85, ws_list_price#86, ws_sold_date_sk#87] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#85), dynamicpruningexpression(ws_sold_date_sk#85 IN dynamicpruning#86)] +PartitionFilters: [isnotnull(ws_sold_date_sk#87), dynamicpruningexpression(ws_sold_date_sk#87 IN dynamicpruning#88)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(85) CometFilter -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 : 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#87] - -(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#88, i_brand_id#89, i_class_id#90, i_category_id#91] - -(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 : 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: 127] -Output [1]: [d_date_sk#92] - -(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 : 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 : 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#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 : 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 : 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 : 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 : 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 : 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#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 : 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#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] +(88) CometFilter +Input [4]: [ws_item_sk#84, ws_quantity#85, ws_list_price#86, ws_sold_date_sk#87] +Condition : isnotnull(ws_item_sk#84) + +(89) ReusedExchange [Reuses operator id: 46] +Output [1]: [ss_item_sk#89] + +(90) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#84, ws_quantity#85, ws_list_price#86, ws_sold_date_sk#87] +Right output [1]: [ss_item_sk#89] +Arguments: [ws_item_sk#84], [ss_item_sk#89], LeftSemi, BuildRight + +(91) ReusedExchange [Reuses operator id: 52] +Output [4]: [i_item_sk#90, i_brand_id#91, i_class_id#92, i_category_id#93] + +(92) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#84, ws_quantity#85, ws_list_price#86, ws_sold_date_sk#87] +Right output [4]: [i_item_sk#90, i_brand_id#91, i_class_id#92, i_category_id#93] +Arguments: [ws_item_sk#84], [i_item_sk#90], Inner, BuildRight + +(93) CometProject +Input [8]: [ws_item_sk#84, ws_quantity#85, ws_list_price#86, ws_sold_date_sk#87, i_item_sk#90, i_brand_id#91, i_class_id#92, i_category_id#93] +Arguments: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#87, i_brand_id#91, i_class_id#92, i_category_id#93], [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#87, i_brand_id#91, i_class_id#92, i_category_id#93] + +(94) ReusedExchange [Reuses operator id: 58] +Output [1]: [d_date_sk#94] + +(95) CometBroadcastHashJoin +Left output [6]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#87, i_brand_id#91, i_class_id#92, i_category_id#93] +Right output [1]: [d_date_sk#94] +Arguments: [ws_sold_date_sk#87], [d_date_sk#94], Inner, BuildRight + +(96) CometProject +Input [7]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#87, i_brand_id#91, i_class_id#92, i_category_id#93, d_date_sk#94] +Arguments: [ws_quantity#85, ws_list_price#86, i_brand_id#91, i_class_id#92, i_category_id#93], [ws_quantity#85, ws_list_price#86, i_brand_id#91, i_class_id#92, i_category_id#93] + +(97) ColumnarToRow [codegen id : 5] +Input [5]: [ws_quantity#85, ws_list_price#86, i_brand_id#91, i_class_id#92, i_category_id#93] + +(98) HashAggregate [codegen id : 5] +Input [5]: [ws_quantity#85, ws_list_price#86, i_brand_id#91, i_class_id#92, i_category_id#93] +Keys [3]: [i_brand_id#91, i_class_id#92, i_category_id#93] +Functions [2]: [partial_sum((cast(ws_quantity#85 as decimal(10,0)) * ws_list_price#86)), partial_count(1)] +Aggregate Attributes [3]: [sum#95, isEmpty#96, count#97] +Results [6]: [i_brand_id#91, i_class_id#92, i_category_id#93, sum#98, isEmpty#99, count#100] + +(99) RowToColumnar +Input [6]: [i_brand_id#91, i_class_id#92, i_category_id#93, sum#98, isEmpty#99, count#100] + +(100) CometColumnarExchange +Input [6]: [i_brand_id#91, i_class_id#92, i_category_id#93, sum#98, isEmpty#99, count#100] +Arguments: hashpartitioning(i_brand_id#91, i_class_id#92, i_category_id#93, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(101) ColumnarToRow [codegen id : 6] +Input [6]: [i_brand_id#91, i_class_id#92, i_category_id#93, sum#98, isEmpty#99, count#100] + +(102) HashAggregate [codegen id : 6] +Input [6]: [i_brand_id#91, i_class_id#92, i_category_id#93, sum#98, isEmpty#99, count#100] +Keys [3]: [i_brand_id#91, i_class_id#92, i_category_id#93] +Functions [2]: [sum((cast(ws_quantity#85 as decimal(10,0)) * ws_list_price#86)), count(1)] +Aggregate Attributes [2]: [sum((cast(ws_quantity#85 as decimal(10,0)) * ws_list_price#86))#101, count(1)#102] +Results [5]: [i_brand_id#91, i_class_id#92, i_category_id#93, sum((cast(ws_quantity#85 as decimal(10,0)) * ws_list_price#86))#101 AS sales#103, count(1)#102 AS number_sales#104] + +(103) Filter [codegen id : 6] +Input [5]: [i_brand_id#91, i_class_id#92, i_category_id#93, sales#103, number_sales#104] +Condition : (isnotnull(sales#103) AND (cast(sales#103 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#56, [id=#57] as decimal(32,6)))) + +(104) Project [codegen id : 6] +Output [6]: [sales#103, number_sales#104, web AS channel#105, i_brand_id#91, i_class_id#92, i_category_id#93] +Input [5]: [i_brand_id#91, i_class_id#92, i_category_id#93, sales#103, number_sales#104] + +(105) Union + +(106) Expand [codegen id : 7] +Input [6]: [sales#54, number_sales#55, channel#58, i_brand_id#59, i_class_id#60, i_category_id#61] +Arguments: [[sales#54, number_sales#55, channel#58, i_brand_id#59, i_class_id#60, i_category_id#61, 0], [sales#54, number_sales#55, channel#58, i_brand_id#59, i_class_id#60, null, 1], [sales#54, number_sales#55, channel#58, i_brand_id#59, null, null, 3], [sales#54, number_sales#55, channel#58, null, null, null, 7], [sales#54, number_sales#55, null, null, null, null, 15]], [sales#54, number_sales#55, channel#106, i_brand_id#107, i_class_id#108, i_category_id#109, spark_grouping_id#110] + +(107) HashAggregate [codegen id : 7] +Input [7]: [sales#54, number_sales#55, channel#106, i_brand_id#107, i_class_id#108, i_category_id#109, spark_grouping_id#110] +Keys [5]: [channel#106, i_brand_id#107, i_class_id#108, i_category_id#109, spark_grouping_id#110] +Functions [2]: [partial_sum(sales#54), partial_sum(number_sales#55)] +Aggregate Attributes [3]: [sum#111, isEmpty#112, sum#113] +Results [8]: [channel#106, i_brand_id#107, i_class_id#108, i_category_id#109, spark_grouping_id#110, sum#114, isEmpty#115, sum#116] + +(108) RowToColumnar +Input [8]: [channel#106, i_brand_id#107, i_class_id#108, i_category_id#109, spark_grouping_id#110, sum#114, isEmpty#115, sum#116] + +(109) CometColumnarExchange +Input [8]: [channel#106, i_brand_id#107, i_class_id#108, i_category_id#109, spark_grouping_id#110, sum#114, isEmpty#115, sum#116] +Arguments: hashpartitioning(channel#106, i_brand_id#107, i_class_id#108, i_category_id#109, spark_grouping_id#110, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(110) ColumnarToRow [codegen id : 8] +Input [8]: [channel#106, i_brand_id#107, i_class_id#108, i_category_id#109, spark_grouping_id#110, sum#114, isEmpty#115, sum#116] + +(111) HashAggregate [codegen id : 8] +Input [8]: [channel#106, i_brand_id#107, i_class_id#108, i_category_id#109, spark_grouping_id#110, sum#114, isEmpty#115, sum#116] +Keys [5]: [channel#106, i_brand_id#107, i_class_id#108, i_category_id#109, spark_grouping_id#110] +Functions [2]: [sum(sales#54), sum(number_sales#55)] +Aggregate Attributes [2]: [sum(sales#54)#117, sum(number_sales#55)#118] +Results [6]: [channel#106, i_brand_id#107, i_class_id#108, i_category_id#109, sum(sales#54)#117 AS sum(sales)#119, sum(number_sales#55)#118 AS sum(number_sales)#120] + +(112) TakeOrderedAndProject +Input [6]: [channel#106, i_brand_id#107, i_class_id#108, i_category_id#109, sum(sales)#119, sum(number_sales)#120] +Arguments: 100, [channel#106 ASC NULLS FIRST, i_brand_id#107 ASC NULLS FIRST, i_class_id#108 ASC NULLS FIRST, i_category_id#109 ASC NULLS FIRST], [channel#106, i_brand_id#107, i_class_id#108, i_category_id#109, sum(sales)#119, sum(number_sales)#120] ===== Subqueries ===== -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#119, ss_list_price#120, ss_sold_date_sk#121] +Subquery:1 Hosting operator id = 67 Hosting Expression = Subquery scalar-subquery#56, [id=#57] +* HashAggregate (131) ++- * ColumnarToRow (130) + +- CometColumnarExchange (129) + +- RowToColumnar (128) + +- * HashAggregate (127) + +- * ColumnarToRow (126) + +- CometUnion (125) + :- CometProject (116) + : +- CometBroadcastHashJoin (115) + : :- CometScan parquet spark_catalog.default.store_sales (113) + : +- ReusedExchange (114) + :- CometProject (120) + : +- CometBroadcastHashJoin (119) + : :- CometScan parquet spark_catalog.default.catalog_sales (117) + : +- ReusedExchange (118) + +- CometProject (124) + +- CometBroadcastHashJoin (123) + :- CometScan parquet spark_catalog.default.web_sales (121) + +- ReusedExchange (122) + + +(113) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_quantity#121, ss_list_price#122, ss_sold_date_sk#123] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#121), dynamicpruningexpression(ss_sold_date_sk#121 IN dynamicpruning#122)] +PartitionFilters: [isnotnull(ss_sold_date_sk#123), dynamicpruningexpression(ss_sold_date_sk#123 IN dynamicpruning#124)] ReadSchema: struct -(107) ReusedExchange [Reuses operator id: 21] -Output [1]: [d_date_sk#123] +(114) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#125] -(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 +(115) CometBroadcastHashJoin +Left output [3]: [ss_quantity#121, ss_list_price#122, ss_sold_date_sk#123] +Right output [1]: [d_date_sk#125] +Arguments: [ss_sold_date_sk#123], [d_date_sk#125], Inner, BuildRight -(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] +(116) CometProject +Input [4]: [ss_quantity#121, ss_list_price#122, ss_sold_date_sk#123, d_date_sk#125] +Arguments: [quantity#126, list_price#127], [ss_quantity#121 AS quantity#126, ss_list_price#122 AS list_price#127] -(110) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#126, cs_list_price#127, cs_sold_date_sk#128] +(117) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#128, cs_list_price#129, cs_sold_date_sk#130] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#128), dynamicpruningexpression(cs_sold_date_sk#128 IN dynamicpruning#129)] +PartitionFilters: [isnotnull(cs_sold_date_sk#130), dynamicpruningexpression(cs_sold_date_sk#130 IN dynamicpruning#131)] ReadSchema: struct -(111) ReusedExchange [Reuses operator id: 21] -Output [1]: [d_date_sk#130] +(118) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#132] -(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 +(119) CometBroadcastHashJoin +Left output [3]: [cs_quantity#128, cs_list_price#129, cs_sold_date_sk#130] +Right output [1]: [d_date_sk#132] +Arguments: [cs_sold_date_sk#130], [d_date_sk#132], Inner, BuildRight -(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] +(120) CometProject +Input [4]: [cs_quantity#128, cs_list_price#129, cs_sold_date_sk#130, d_date_sk#132] +Arguments: [quantity#133, list_price#134], [cs_quantity#128 AS quantity#133, cs_list_price#129 AS list_price#134] -(114) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#133, ws_list_price#134, ws_sold_date_sk#135] +(121) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#135, ws_list_price#136, ws_sold_date_sk#137] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#135), dynamicpruningexpression(ws_sold_date_sk#135 IN dynamicpruning#136)] +PartitionFilters: [isnotnull(ws_sold_date_sk#137), dynamicpruningexpression(ws_sold_date_sk#137 IN dynamicpruning#138)] ReadSchema: struct -(115) ReusedExchange [Reuses operator id: 21] -Output [1]: [d_date_sk#137] +(122) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#139] -(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 +(123) CometBroadcastHashJoin +Left output [3]: [ws_quantity#135, ws_list_price#136, ws_sold_date_sk#137] +Right output [1]: [d_date_sk#139] +Arguments: [ws_sold_date_sk#137], [d_date_sk#139], Inner, BuildRight -(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] +(124) CometProject +Input [4]: [ws_quantity#135, ws_list_price#136, ws_sold_date_sk#137, d_date_sk#139] +Arguments: [quantity#140, list_price#141], [ws_quantity#135 AS quantity#140, ws_list_price#136 AS list_price#141] -(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] +(125) CometUnion +Child 0 Input [2]: [quantity#126, list_price#127] +Child 1 Input [2]: [quantity#133, list_price#134] +Child 2 Input [2]: [quantity#140, list_price#141] -(119) ColumnarToRow [codegen id : 1] -Input [2]: [quantity#124, list_price#125] +(126) ColumnarToRow [codegen id : 1] +Input [2]: [quantity#126, list_price#127] -(120) HashAggregate [codegen id : 1] -Input [2]: [quantity#124, list_price#125] +(127) HashAggregate [codegen id : 1] +Input [2]: [quantity#126, list_price#127] Keys: [] -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] +Functions [1]: [partial_avg((cast(quantity#126 as decimal(10,0)) * list_price#127))] +Aggregate Attributes [2]: [sum#142, count#143] +Results [2]: [sum#144, count#145] + +(128) RowToColumnar +Input [2]: [sum#144, count#145] -(121) Exchange -Input [2]: [sum#142, count#143] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] +(129) CometColumnarExchange +Input [2]: [sum#144, count#145] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(122) HashAggregate [codegen id : 2] -Input [2]: [sum#142, count#143] +(130) ColumnarToRow [codegen id : 2] +Input [2]: [sum#144, count#145] + +(131) HashAggregate [codegen id : 2] +Input [2]: [sum#144, count#145] Keys: [] -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] +Functions [1]: [avg((cast(quantity#126 as decimal(10,0)) * list_price#127))] +Aggregate Attributes [1]: [avg((cast(quantity#126 as decimal(10,0)) * list_price#127))#146] +Results [1]: [avg((cast(quantity#126 as decimal(10,0)) * list_price#127))#146 AS average_sales#147] -Subquery:2 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#121 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 113 Hosting Expression = ss_sold_date_sk#123 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#128 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 117 Hosting Expression = cs_sold_date_sk#130 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 114 Hosting Expression = ws_sold_date_sk#135 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 121 Hosting Expression = ws_sold_date_sk#137 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (127) -+- * ColumnarToRow (126) - +- CometProject (125) - +- CometFilter (124) - +- CometScan parquet spark_catalog.default.date_dim (123) +BroadcastExchange (136) ++- * ColumnarToRow (135) + +- CometProject (134) + +- CometFilter (133) + +- CometScan parquet spark_catalog.default.date_dim (132) -(123) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#43, d_year#146, d_moy#147] +(132) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#43, d_year#44, d_moy#45] 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 -(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)) +(133) CometFilter +Input [3]: [d_date_sk#43, 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 = 11)) AND isnotnull(d_date_sk#43)) -(125) CometProject -Input [3]: [d_date_sk#43, d_year#146, d_moy#147] +(134) CometProject +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] Arguments: [d_date_sk#43], [d_date_sk#43] -(126) ColumnarToRow [codegen id : 1] +(135) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] -(127) BroadcastExchange +(136) BroadcastExchange Input [1]: [d_date_sk#43] -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=7] -Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (132) -+- * ColumnarToRow (131) - +- CometProject (130) - +- CometFilter (129) - +- CometScan parquet spark_catalog.default.date_dim (128) +Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 +BroadcastExchange (141) ++- * ColumnarToRow (140) + +- CometProject (139) + +- CometFilter (138) + +- CometScan parquet spark_catalog.default.date_dim (137) -(128) Scan parquet spark_catalog.default.date_dim +(137) 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 -(129) CometFilter +(138) 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)) -(130) CometProject +(139) CometProject Input [2]: [d_date_sk#26, d_year#148] Arguments: [d_date_sk#26], [d_date_sk#26] -(131) ColumnarToRow [codegen id : 1] +(140) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] -(132) BroadcastExchange +(141) BroadcastExchange Input [1]: [d_date_sk#26] -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=8] -Subquery:7 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 +Subquery:7 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 82 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] +Subquery:9 Hosting operator id = 85 Hosting Expression = ReusedSubquery Subquery scalar-subquery#56, [id=#57] -Subquery:10 Hosting operator id = 68 Hosting Expression = cs_sold_date_sk#63 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#65 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 98 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] +Subquery:11 Hosting operator id = 103 Hosting Expression = ReusedSubquery Subquery scalar-subquery#56, [id=#57] -Subquery:12 Hosting operator id = 84 Hosting Expression = ws_sold_date_sk#85 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 87 Hosting Expression = ws_sold_date_sk#87 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 d2ea2d830..9b8f32e55 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,190 +1,177 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] - WholeStageCodegen (38) + WholeStageCodegen (8) 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 (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 (12) - Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] - Filter [sales] - Subquery #3 - WholeStageCodegen (2) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - InputAdapter - Exchange #14 - WholeStageCodegen (1) - HashAggregate [quantity,list_price] [sum,count,sum,count] - ColumnarToRow - InputAdapter - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,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_quantity,cs_list_price,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_quantity,ws_list_price,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 (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] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 + RowToColumnar + WholeStageCodegen (7) + 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 (2) + Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] + Filter [sales] + Subquery #3 + WholeStageCodegen (2) + HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] + ColumnarToRow + InputAdapter + CometColumnarExchange #15 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [quantity,list_price] [sum,count,sum,count] + ColumnarToRow + InputAdapter + CometUnion [quantity,list_price] + CometProject [ss_quantity,ss_list_price] [quantity,list_price] + CometBroadcastHashJoin [ss_quantity,ss_list_price,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_quantity,cs_list_price,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_quantity,ws_list_price,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] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #2 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - ColumnarToRow - InputAdapter + CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk] #4 + CometProject [i_item_sk] [ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,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 [brand_id,class_id,category_id] #5 + CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + CometHashAggregate [brand_id,class_id,category_id] + CometColumnarExchange [brand_id,class_id,category_id] #6 + 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,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8 + CometBroadcastHashJoin [i_item_sk,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 [i_brand_id,i_class_id,i_category_id] #9 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + 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 [d_date_sk] #11 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #11 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #12 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + ReusedExchange [d_date_sk] #11 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] 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 #5 - 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 (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,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8 - CometBroadcastHashJoin [i_item_sk,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 [i_brand_id,i_class_id,i_category_id] #9 - CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [cs_item_sk,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 - 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 [d_date_sk] #11 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #11 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ws_item_sk,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 - ReusedExchange [d_date_sk] #11 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (9) - BroadcastHashJoin [i_item_sk,ss_item_sk] - 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 - ReusedExchange [ss_item_sk] #4 - InputAdapter - ReusedExchange [d_date_sk] #3 - 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] #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] - Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - BroadcastHashJoin [cs_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ReusedExchange [ss_item_sk] #4 + CometBroadcastExchange [d_date_sk] #14 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + WholeStageCodegen (4) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #16 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] + ColumnarToRow InputAdapter - ReusedExchange [ss_item_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - InputAdapter - ReusedExchange [d_date_sk] #3 - 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] #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] - Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - BroadcastHashJoin [ws_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + CometProject [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,ss_item_sk] + CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [ss_item_sk] #4 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + ReusedExchange [d_date_sk] #14 + WholeStageCodegen (6) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #17 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] + ColumnarToRow InputAdapter - ReusedExchange [ss_item_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - InputAdapter - ReusedExchange [d_date_sk] #3 + CometProject [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,ss_item_sk] + CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [ss_item_sk] #4 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + ReusedExchange [d_date_sk] #14 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 842435d19..ce4cf849b 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 @@ -1,88 +1,94 @@ == Physical Plan == -TakeOrderedAndProject (84) -+- * BroadcastHashJoin Inner BuildRight (83) - :- * Filter (66) - : +- * HashAggregate (65) - : +- Exchange (64) - : +- * HashAggregate (63) - : +- * Project (62) - : +- * BroadcastHashJoin Inner BuildRight (61) - : :- * Project (59) - : : +- * BroadcastHashJoin Inner BuildRight (58) - : : :- * BroadcastHashJoin LeftSemi BuildRight (51) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (50) - : : : +- * Project (49) - : : : +- * BroadcastHashJoin Inner BuildRight (48) - : : : :- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.item (4) - : : : +- BroadcastExchange (47) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) - : : : :- * HashAggregate (35) - : : : : +- Exchange (34) - : : : : +- * 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) - : : : +- * 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) - : : : +- CometFilter (53) - : : : +- CometScan parquet spark_catalog.default.item (52) - : : +- ReusedExchange (55) - : +- ReusedExchange (60) - +- BroadcastExchange (82) - +- * Filter (81) - +- * HashAggregate (80) - +- Exchange (79) - +- * HashAggregate (78) - +- * Project (77) - +- * BroadcastHashJoin Inner BuildRight (76) - :- * Project (74) - : +- * BroadcastHashJoin Inner BuildRight (73) - : :- * BroadcastHashJoin LeftSemi BuildRight (71) - : : :- * ColumnarToRow (69) - : : : +- CometFilter (68) - : : : +- CometScan parquet spark_catalog.default.store_sales (67) - : : +- ReusedExchange (70) - : +- ReusedExchange (72) - +- ReusedExchange (75) +TakeOrderedAndProject (90) ++- * BroadcastHashJoin Inner BuildRight (89) + :- * Filter (67) + : +- * HashAggregate (66) + : +- * ColumnarToRow (65) + : +- CometColumnarExchange (64) + : +- RowToColumnar (63) + : +- * HashAggregate (62) + : +- * ColumnarToRow (61) + : +- CometProject (60) + : +- CometBroadcastHashJoin (59) + : :- CometProject (54) + : : +- CometBroadcastHashJoin (53) + : : :- CometBroadcastHashJoin (47) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (46) + : : : +- CometProject (45) + : : : +- CometBroadcastHashJoin (44) + : : : :- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.item (3) + : : : +- CometBroadcastExchange (43) + : : : +- CometBroadcastHashJoin (42) + : : : :- CometHashAggregate (32) + : : : : +- CometColumnarExchange (31) + : : : : +- CometHashAggregate (30) + : : : : +- CometProject (29) + : : : : +- CometBroadcastHashJoin (28) + : : : : :- CometProject (26) + : : : : : +- CometBroadcastHashJoin (25) + : : : : : :- CometFilter (6) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (5) + : : : : : +- CometBroadcastExchange (24) + : : : : : +- CometBroadcastHashJoin (23) + : : : : : :- CometFilter (8) + : : : : : : +- CometScan parquet spark_catalog.default.item (7) + : : : : : +- CometBroadcastExchange (22) + : : : : : +- CometProject (21) + : : : : : +- CometBroadcastHashJoin (20) + : : : : : :- CometProject (15) + : : : : : : +- CometBroadcastHashJoin (14) + : : : : : : :- CometFilter (10) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (9) + : : : : : : +- CometBroadcastExchange (13) + : : : : : : +- CometFilter (12) + : : : : : : +- CometScan parquet spark_catalog.default.item (11) + : : : : : +- CometBroadcastExchange (19) + : : : : : +- CometProject (18) + : : : : : +- CometFilter (17) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (16) + : : : : +- ReusedExchange (27) + : : : +- CometBroadcastExchange (41) + : : : +- CometProject (40) + : : : +- CometBroadcastHashJoin (39) + : : : :- CometProject (37) + : : : : +- CometBroadcastHashJoin (36) + : : : : :- CometFilter (34) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (33) + : : : : +- ReusedExchange (35) + : : : +- ReusedExchange (38) + : : +- CometBroadcastExchange (52) + : : +- CometBroadcastHashJoin (51) + : : :- CometFilter (49) + : : : +- CometScan parquet spark_catalog.default.item (48) + : : +- ReusedExchange (50) + : +- CometBroadcastExchange (58) + : +- CometProject (57) + : +- CometFilter (56) + : +- CometScan parquet spark_catalog.default.date_dim (55) + +- BroadcastExchange (88) + +- * Filter (87) + +- * HashAggregate (86) + +- * ColumnarToRow (85) + +- CometColumnarExchange (84) + +- RowToColumnar (83) + +- * HashAggregate (82) + +- * ColumnarToRow (81) + +- CometProject (80) + +- CometBroadcastHashJoin (79) + :- CometProject (74) + : +- CometBroadcastHashJoin (73) + : :- CometBroadcastHashJoin (71) + : : :- CometFilter (69) + : : : +- CometScan parquet spark_catalog.default.store_sales (68) + : : +- ReusedExchange (70) + : +- ReusedExchange (72) + +- CometBroadcastExchange (78) + +- CometProject (77) + +- CometFilter (76) + +- CometScan parquet spark_catalog.default.date_dim (75) (1) Scan parquet spark_catalog.default.store_sales @@ -97,24 +103,18 @@ 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 : 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 +(3) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) -(6) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(7) Scan parquet spark_catalog.default.store_sales +(5) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] @@ -122,22 +122,22 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(8) CometFilter +(6) CometFilter Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(9) Scan parquet spark_catalog.default.item +(7) 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 -(10) CometFilter +(8) 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)) -(11) Scan parquet spark_catalog.default.catalog_sales +(9) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] @@ -145,116 +145,111 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(12) CometFilter +(10) CometFilter Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) -(13) Scan parquet spark_catalog.default.item +(11) 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 -(14) CometFilter +(12) 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) -(15) CometBroadcastExchange +(13) 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] -(16) CometBroadcastHashJoin +(14) 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 -(17) CometProject +(15) 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 +(16) 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 -(19) CometFilter +(17) 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 +(18) CometProject Input [2]: [d_date_sk#24, d_year#25] Arguments: [d_date_sk#24], [d_date_sk#24] -(21) CometBroadcastExchange +(19) CometBroadcastExchange Input [1]: [d_date_sk#24] Arguments: [d_date_sk#24] -(22) CometBroadcastHashJoin +(20) 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 +(21) 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] -(24) CometBroadcastExchange +(22) CometBroadcastExchange Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] -(25) CometBroadcastHashJoin +(23) 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 -(26) CometBroadcastExchange +(24) CometBroadcastExchange Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(27) CometBroadcastHashJoin +(25) 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 -(28) CometProject +(26) 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] -(29) ReusedExchange [Reuses operator id: 21] +(27) ReusedExchange [Reuses operator id: 19] Output [1]: [d_date_sk#26] -(30) CometBroadcastHashJoin +(28) 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 -(31) CometProject +(29) 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] -(32) CometHashAggregate +(30) CometHashAggregate Input [3]: [brand_id#27, class_id#28, category_id#29] Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] -(33) ColumnarToRow [codegen id : 1] +(31) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(34) Exchange -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 : 3] +(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#27, class_id#28, category_id#29] -(36) Scan parquet spark_catalog.default.web_sales +(33) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] @@ -262,488 +257,536 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(37) CometFilter +(34) CometFilter Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] Condition : isnotnull(ws_item_sk#30) -(38) ReusedExchange [Reuses operator id: 15] +(35) ReusedExchange [Reuses operator id: 13] Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -(39) CometBroadcastHashJoin +(36) 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) CometProject +(37) 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) ReusedExchange [Reuses operator id: 21] +(38) ReusedExchange [Reuses operator id: 19] Output [1]: [d_date_sk#37] -(42) CometBroadcastHashJoin +(39) 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) CometProject +(40) 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) ColumnarToRow [codegen id : 2] +(41) CometBroadcastExchange Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] -(45) BroadcastExchange -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 : 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 +(42) CometBroadcastHashJoin +Left output [3]: [brand_id#27, class_id#28, category_id#29] +Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [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)], [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)], LeftSemi, BuildRight -(47) BroadcastExchange +(43) CometBroadcastExchange 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] +Arguments: [brand_id#27, class_id#28, category_id#29] -(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#27, class_id#28, category_id#29] -Join type: Inner -Join condition: None +(44) CometBroadcastHashJoin +Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Right output [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight -(49) Project [codegen id : 4] -Output [1]: [i_item_sk#6 AS ss_item_sk#38] +(45) CometProject 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] +Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] -(50) BroadcastExchange +(46) CometBroadcastExchange Input [1]: [ss_item_sk#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: [ss_item_sk#38] -(51) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#38] -Join type: LeftSemi -Join condition: None +(47) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Right output [1]: [ss_item_sk#38] +Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight -(52) Scan parquet spark_catalog.default.item +(48) Scan parquet spark_catalog.default.item 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 +(49) CometFilter 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 : 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] +(50) ReusedExchange [Reuses operator id: 46] Output [1]: [ss_item_sk#38] -(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 +(51) CometBroadcastHashJoin +Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Right output [1]: [ss_item_sk#38] +Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight -(57) BroadcastExchange +(52) CometBroadcastExchange 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] +Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -(58) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#39] -Join type: Inner -Join condition: None +(53) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight -(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] +(54) CometProject 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] +Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] + +(55) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#43, d_week_seq#44] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#45), IsNotNull(d_date_sk)] +ReadSchema: struct -(60) ReusedExchange [Reuses operator id: 106] -Output [1]: [d_date_sk#43] +(56) CometFilter +Input [2]: [d_date_sk#43, d_week_seq#44] +Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = ReusedSubquery Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) -(61) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#43] -Join type: Inner -Join condition: None +(57) CometProject +Input [2]: [d_date_sk#43, d_week_seq#44] +Arguments: [d_date_sk#43], [d_date_sk#43] -(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] +(58) CometBroadcastExchange +Input [1]: [d_date_sk#43] +Arguments: [d_date_sk#43] + +(59) CometBroadcastHashJoin +Left 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] +Right output [1]: [d_date_sk#43] +Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight + +(60) CometProject 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] +Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] + +(61) ColumnarToRow [codegen id : 1] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -(63) HashAggregate [codegen id : 11] +(62) HashAggregate [codegen id : 1] 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#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] +Aggregate Attributes [3]: [sum#47, isEmpty#48, count#49] +Results [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#50, isEmpty#51, count#52] -(64) Exchange -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] +(63) RowToColumnar +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#50, isEmpty#51, count#52] -(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] +(64) CometColumnarExchange +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#50, isEmpty#51, count#52] +Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(65) ColumnarToRow [codegen id : 4] +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#50, isEmpty#51, count#52] + +(66) HashAggregate [codegen id : 4] +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#50, isEmpty#51, count#52] 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))#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] +Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#53, count(1)#54] +Results [6]: [store AS channel#55, 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))#53 AS sales#56, count(1)#54 AS number_sales#57] -(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) Filter [codegen id : 4] +Input [6]: [channel#55, i_brand_id#40, i_class_id#41, i_category_id#42, sales#56, number_sales#57] +Condition : (isnotnull(sales#56) AND (cast(sales#56 as decimal(32,6)) > cast(Subquery scalar-subquery#58, [id=#59] as decimal(32,6)))) -(67) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60] +(68) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_item_sk#60, ss_quantity#61, ss_list_price#62, ss_sold_date_sk#63] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#60), dynamicpruningexpression(ss_sold_date_sk#60 IN dynamicpruning#61)] +PartitionFilters: [isnotnull(ss_sold_date_sk#63), dynamicpruningexpression(ss_sold_date_sk#63 IN dynamicpruning#64)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(68) CometFilter -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) CometFilter +Input [4]: [ss_item_sk#60, ss_quantity#61, ss_list_price#62, ss_sold_date_sk#63] +Condition : isnotnull(ss_item_sk#60) -(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: 46] +Output [1]: [ss_item_sk#65] -(70) ReusedExchange [Reuses operator id: 50] -Output [1]: [ss_item_sk#62] +(71) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#60, ss_quantity#61, ss_list_price#62, ss_sold_date_sk#63] +Right output [1]: [ss_item_sk#65] +Arguments: [ss_item_sk#60], [ss_item_sk#65], LeftSemi, BuildRight -(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: 52] +Output [4]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69] -(72) ReusedExchange [Reuses operator id: 57] -Output [4]: [i_item_sk#63, i_brand_id#64, i_class_id#65, i_category_id#66] +(73) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#60, ss_quantity#61, ss_list_price#62, ss_sold_date_sk#63] +Right output [4]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69] +Arguments: [ss_item_sk#60], [i_item_sk#66], Inner, BuildRight -(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) CometProject +Input [8]: [ss_item_sk#60, ss_quantity#61, ss_list_price#62, ss_sold_date_sk#63, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69] +Arguments: [ss_quantity#61, ss_list_price#62, ss_sold_date_sk#63, i_brand_id#67, i_class_id#68, i_category_id#69], [ss_quantity#61, ss_list_price#62, ss_sold_date_sk#63, i_brand_id#67, i_class_id#68, i_category_id#69] -(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) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#70, d_week_seq#71] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#72), IsNotNull(d_date_sk)] +ReadSchema: struct -(75) ReusedExchange [Reuses operator id: 120] -Output [1]: [d_date_sk#67] +(76) CometFilter +Input [2]: [d_date_sk#70, d_week_seq#71] +Condition : ((isnotnull(d_week_seq#71) AND (d_week_seq#71 = ReusedSubquery Subquery scalar-subquery#72, [id=#73])) AND isnotnull(d_date_sk#70)) -(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) CometProject +Input [2]: [d_date_sk#70, d_week_seq#71] +Arguments: [d_date_sk#70], [d_date_sk#70] + +(78) CometBroadcastExchange +Input [1]: [d_date_sk#70] +Arguments: [d_date_sk#70] + +(79) CometBroadcastHashJoin +Left output [6]: [ss_quantity#61, ss_list_price#62, ss_sold_date_sk#63, i_brand_id#67, i_class_id#68, i_category_id#69] +Right output [1]: [d_date_sk#70] +Arguments: [ss_sold_date_sk#63], [d_date_sk#70], Inner, BuildRight + +(80) CometProject +Input [7]: [ss_quantity#61, ss_list_price#62, ss_sold_date_sk#63, i_brand_id#67, i_class_id#68, i_category_id#69, d_date_sk#70] +Arguments: [ss_quantity#61, ss_list_price#62, i_brand_id#67, i_class_id#68, i_category_id#69], [ss_quantity#61, ss_list_price#62, i_brand_id#67, i_class_id#68, i_category_id#69] + +(81) ColumnarToRow [codegen id : 2] +Input [5]: [ss_quantity#61, ss_list_price#62, i_brand_id#67, i_class_id#68, i_category_id#69] + +(82) HashAggregate [codegen id : 2] +Input [5]: [ss_quantity#61, ss_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(ss_quantity#61 as decimal(10,0)) * ss_list_price#62)), partial_count(1)] +Aggregate Attributes [3]: [sum#74, isEmpty#75, count#76] +Results [6]: [i_brand_id#67, i_class_id#68, i_category_id#69, sum#77, isEmpty#78, count#79] + +(83) RowToColumnar +Input [6]: [i_brand_id#67, i_class_id#68, i_category_id#69, sum#77, isEmpty#78, count#79] + +(84) CometColumnarExchange +Input [6]: [i_brand_id#67, i_class_id#68, i_category_id#69, sum#77, isEmpty#78, count#79] +Arguments: hashpartitioning(i_brand_id#67, i_class_id#68, i_category_id#69, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(85) ColumnarToRow [codegen id : 3] +Input [6]: [i_brand_id#67, i_class_id#68, i_category_id#69, sum#77, isEmpty#78, count#79] -(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 : 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#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 : 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 : 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#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 : 24] +(86) HashAggregate [codegen id : 3] +Input [6]: [i_brand_id#67, i_class_id#68, i_category_id#69, sum#77, isEmpty#78, count#79] +Keys [3]: [i_brand_id#67, i_class_id#68, i_category_id#69] +Functions [2]: [sum((cast(ss_quantity#61 as decimal(10,0)) * ss_list_price#62)), count(1)] +Aggregate Attributes [2]: [sum((cast(ss_quantity#61 as decimal(10,0)) * ss_list_price#62))#80, count(1)#81] +Results [6]: [store AS channel#82, i_brand_id#67, i_class_id#68, i_category_id#69, sum((cast(ss_quantity#61 as decimal(10,0)) * ss_list_price#62))#80 AS sales#83, count(1)#81 AS number_sales#84] + +(87) Filter [codegen id : 3] +Input [6]: [channel#82, i_brand_id#67, i_class_id#68, i_category_id#69, sales#83, number_sales#84] +Condition : (isnotnull(sales#83) AND (cast(sales#83 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#58, [id=#59] as decimal(32,6)))) + +(88) BroadcastExchange +Input [6]: [channel#82, i_brand_id#67, i_class_id#68, i_category_id#69, sales#83, number_sales#84] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=4] + +(89) BroadcastHashJoin [codegen id : 4] 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] +Right keys [3]: [i_brand_id#67, i_class_id#68, i_category_id#69] Join type: Inner Join condition: None -(84) TakeOrderedAndProject -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] +(90) TakeOrderedAndProject +Input [12]: [channel#55, i_brand_id#40, i_class_id#41, i_category_id#42, sales#56, number_sales#57, channel#82, i_brand_id#67, i_class_id#68, i_category_id#69, sales#83, number_sales#84] +Arguments: 100, [i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#55, i_brand_id#40, i_class_id#41, i_category_id#42, sales#56, number_sales#57, channel#82, i_brand_id#67, i_class_id#68, i_category_id#69, sales#83, number_sales#84] ===== Subqueries ===== -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#79, ss_list_price#80, ss_sold_date_sk#81] +Subquery:1 Hosting operator id = 67 Hosting Expression = Subquery scalar-subquery#58, [id=#59] +* HashAggregate (109) ++- * ColumnarToRow (108) + +- CometColumnarExchange (107) + +- RowToColumnar (106) + +- * HashAggregate (105) + +- * ColumnarToRow (104) + +- CometUnion (103) + :- CometProject (94) + : +- CometBroadcastHashJoin (93) + : :- CometScan parquet spark_catalog.default.store_sales (91) + : +- ReusedExchange (92) + :- CometProject (98) + : +- CometBroadcastHashJoin (97) + : :- CometScan parquet spark_catalog.default.catalog_sales (95) + : +- ReusedExchange (96) + +- CometProject (102) + +- CometBroadcastHashJoin (101) + :- CometScan parquet spark_catalog.default.web_sales (99) + +- ReusedExchange (100) + + +(91) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_quantity#85, ss_list_price#86, ss_sold_date_sk#87] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#81), dynamicpruningexpression(ss_sold_date_sk#81 IN dynamicpruning#82)] +PartitionFilters: [isnotnull(ss_sold_date_sk#87), dynamicpruningexpression(ss_sold_date_sk#87 IN dynamicpruning#88)] ReadSchema: struct -(86) ReusedExchange [Reuses operator id: 21] -Output [1]: [d_date_sk#83] +(92) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#89] -(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 +(93) CometBroadcastHashJoin +Left output [3]: [ss_quantity#85, ss_list_price#86, ss_sold_date_sk#87] +Right output [1]: [d_date_sk#89] +Arguments: [ss_sold_date_sk#87], [d_date_sk#89], Inner, BuildRight -(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] +(94) CometProject +Input [4]: [ss_quantity#85, ss_list_price#86, ss_sold_date_sk#87, d_date_sk#89] +Arguments: [quantity#90, list_price#91], [ss_quantity#85 AS quantity#90, ss_list_price#86 AS list_price#91] -(89) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#86, cs_list_price#87, cs_sold_date_sk#88] +(95) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#92, cs_list_price#93, cs_sold_date_sk#94] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#88), dynamicpruningexpression(cs_sold_date_sk#88 IN dynamicpruning#89)] +PartitionFilters: [isnotnull(cs_sold_date_sk#94), dynamicpruningexpression(cs_sold_date_sk#94 IN dynamicpruning#95)] ReadSchema: struct -(90) ReusedExchange [Reuses operator id: 21] -Output [1]: [d_date_sk#90] +(96) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#96] -(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 +(97) CometBroadcastHashJoin +Left output [3]: [cs_quantity#92, cs_list_price#93, cs_sold_date_sk#94] +Right output [1]: [d_date_sk#96] +Arguments: [cs_sold_date_sk#94], [d_date_sk#96], Inner, BuildRight -(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] +(98) CometProject +Input [4]: [cs_quantity#92, cs_list_price#93, cs_sold_date_sk#94, d_date_sk#96] +Arguments: [quantity#97, list_price#98], [cs_quantity#92 AS quantity#97, cs_list_price#93 AS list_price#98] -(93) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#93, ws_list_price#94, ws_sold_date_sk#95] +(99) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#99, ws_list_price#100, ws_sold_date_sk#101] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#95), dynamicpruningexpression(ws_sold_date_sk#95 IN dynamicpruning#96)] +PartitionFilters: [isnotnull(ws_sold_date_sk#101), dynamicpruningexpression(ws_sold_date_sk#101 IN dynamicpruning#102)] ReadSchema: struct -(94) ReusedExchange [Reuses operator id: 21] -Output [1]: [d_date_sk#97] +(100) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#103] -(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 +(101) CometBroadcastHashJoin +Left output [3]: [ws_quantity#99, ws_list_price#100, ws_sold_date_sk#101] +Right output [1]: [d_date_sk#103] +Arguments: [ws_sold_date_sk#101], [d_date_sk#103], Inner, BuildRight -(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] +(102) CometProject +Input [4]: [ws_quantity#99, ws_list_price#100, ws_sold_date_sk#101, d_date_sk#103] +Arguments: [quantity#104, list_price#105], [ws_quantity#99 AS quantity#104, ws_list_price#100 AS list_price#105] -(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] +(103) CometUnion +Child 0 Input [2]: [quantity#90, list_price#91] +Child 1 Input [2]: [quantity#97, list_price#98] +Child 2 Input [2]: [quantity#104, list_price#105] -(98) ColumnarToRow [codegen id : 1] -Input [2]: [quantity#84, list_price#85] +(104) ColumnarToRow [codegen id : 1] +Input [2]: [quantity#90, list_price#91] -(99) HashAggregate [codegen id : 1] -Input [2]: [quantity#84, list_price#85] +(105) HashAggregate [codegen id : 1] +Input [2]: [quantity#90, list_price#91] Keys: [] -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] +Functions [1]: [partial_avg((cast(quantity#90 as decimal(10,0)) * list_price#91))] +Aggregate Attributes [2]: [sum#106, count#107] +Results [2]: [sum#108, count#109] + +(106) RowToColumnar +Input [2]: [sum#108, count#109] + +(107) CometColumnarExchange +Input [2]: [sum#108, count#109] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(100) Exchange -Input [2]: [sum#102, count#103] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] +(108) ColumnarToRow [codegen id : 2] +Input [2]: [sum#108, count#109] -(101) HashAggregate [codegen id : 2] -Input [2]: [sum#102, count#103] +(109) HashAggregate [codegen id : 2] +Input [2]: [sum#108, count#109] Keys: [] -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] +Functions [1]: [avg((cast(quantity#90 as decimal(10,0)) * list_price#91))] +Aggregate Attributes [1]: [avg((cast(quantity#90 as decimal(10,0)) * list_price#91))#110] +Results [1]: [avg((cast(quantity#90 as decimal(10,0)) * list_price#91))#110 AS average_sales#111] -Subquery:2 Hosting operator id = 85 Hosting Expression = ss_sold_date_sk#81 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 89 Hosting Expression = cs_sold_date_sk#88 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 95 Hosting Expression = cs_sold_date_sk#94 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 93 Hosting Expression = ws_sold_date_sk#95 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 99 Hosting Expression = ws_sold_date_sk#101 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (106) -+- * ColumnarToRow (105) - +- CometProject (104) - +- CometFilter (103) - +- CometScan parquet spark_catalog.default.date_dim (102) +BroadcastExchange (114) ++- * ColumnarToRow (113) + +- CometProject (112) + +- CometFilter (111) + +- CometScan parquet spark_catalog.default.date_dim (110) -(102) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_week_seq#106] +(110) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#43, d_week_seq#44] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#107), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#45), IsNotNull(d_date_sk)] ReadSchema: struct -(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)) +(111) CometFilter +Input [2]: [d_date_sk#43, d_week_seq#44] +Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = ReusedSubquery Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) -(104) CometProject -Input [2]: [d_date_sk#43, d_week_seq#106] +(112) CometProject +Input [2]: [d_date_sk#43, d_week_seq#44] Arguments: [d_date_sk#43], [d_date_sk#43] -(105) ColumnarToRow [codegen id : 1] +(113) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] -(106) BroadcastExchange +(114) BroadcastExchange Input [1]: [d_date_sk#43] -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=6] -Subquery:6 Hosting operator id = 103 Hosting Expression = ReusedSubquery Subquery scalar-subquery#107, [id=#108] +Subquery:6 Hosting operator id = 111 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -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) +Subquery:7 Hosting operator id = 110 Hosting Expression = Subquery scalar-subquery#45, [id=#46] +* ColumnarToRow (118) ++- CometProject (117) + +- CometFilter (116) + +- CometScan parquet spark_catalog.default.date_dim (115) -(107) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#109, d_year#110, d_moy#111, d_dom#112] +(115) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_week_seq#112, d_year#113, d_moy#114, d_dom#115] 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 -(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)) +(116) CometFilter +Input [4]: [d_week_seq#112, d_year#113, d_moy#114, d_dom#115] +Condition : (((((isnotnull(d_year#113) AND isnotnull(d_moy#114)) AND isnotnull(d_dom#115)) AND (d_year#113 = 2000)) AND (d_moy#114 = 12)) AND (d_dom#115 = 11)) -(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] +(117) CometProject +Input [4]: [d_week_seq#112, d_year#113, d_moy#114, d_dom#115] +Arguments: [d_week_seq#112], [d_week_seq#112] -(110) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#109] +(118) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#112] -Subquery:8 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (115) -+- * ColumnarToRow (114) - +- CometProject (113) - +- CometFilter (112) - +- CometScan parquet spark_catalog.default.date_dim (111) +Subquery:8 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 +BroadcastExchange (123) ++- * ColumnarToRow (122) + +- CometProject (121) + +- CometFilter (120) + +- CometScan parquet spark_catalog.default.date_dim (119) -(111) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#113] +(119) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_year#116] 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 -(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)) +(120) CometFilter +Input [2]: [d_date_sk#26, d_year#116] +Condition : (((isnotnull(d_year#116) AND (d_year#116 >= 1999)) AND (d_year#116 <= 2001)) AND isnotnull(d_date_sk#26)) -(113) CometProject -Input [2]: [d_date_sk#26, d_year#113] +(121) CometProject +Input [2]: [d_date_sk#26, d_year#116] Arguments: [d_date_sk#26], [d_date_sk#26] -(114) ColumnarToRow [codegen id : 1] +(122) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] -(115) BroadcastExchange +(123) BroadcastExchange Input [1]: [d_date_sk#26] -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=7] -Subquery:9 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 +Subquery:9 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:10 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 +Subquery:10 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 -Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#55, [id=#56] +Subquery:11 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -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) +Subquery:12 Hosting operator id = 55 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:13 Hosting operator id = 87 Hosting Expression = ReusedSubquery Subquery scalar-subquery#58, [id=#59] -(116) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#67, d_week_seq#114] +Subquery:14 Hosting operator id = 68 Hosting Expression = ss_sold_date_sk#63 IN dynamicpruning#64 +BroadcastExchange (128) ++- * ColumnarToRow (127) + +- CometProject (126) + +- CometFilter (125) + +- CometScan parquet spark_catalog.default.date_dim (124) + + +(124) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#70, d_week_seq#71] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#115), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#72), IsNotNull(d_date_sk)] ReadSchema: struct -(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)) +(125) CometFilter +Input [2]: [d_date_sk#70, d_week_seq#71] +Condition : ((isnotnull(d_week_seq#71) AND (d_week_seq#71 = ReusedSubquery Subquery scalar-subquery#72, [id=#73])) AND isnotnull(d_date_sk#70)) -(118) CometProject -Input [2]: [d_date_sk#67, d_week_seq#114] -Arguments: [d_date_sk#67], [d_date_sk#67] +(126) CometProject +Input [2]: [d_date_sk#70, d_week_seq#71] +Arguments: [d_date_sk#70], [d_date_sk#70] -(119) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#67] +(127) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#70] -(120) BroadcastExchange -Input [1]: [d_date_sk#67] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] +(128) BroadcastExchange +Input [1]: [d_date_sk#70] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:13 Hosting operator id = 117 Hosting Expression = ReusedSubquery Subquery scalar-subquery#115, [id=#116] +Subquery:15 Hosting operator id = 125 Hosting Expression = ReusedSubquery Subquery scalar-subquery#72, [id=#73] -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) +Subquery:16 Hosting operator id = 124 Hosting Expression = Subquery scalar-subquery#72, [id=#73] +* ColumnarToRow (132) ++- CometProject (131) + +- CometFilter (130) + +- CometScan parquet spark_catalog.default.date_dim (129) -(121) Scan parquet spark_catalog.default.date_dim +(129) 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 -(122) CometFilter +(130) 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)) -(123) CometProject +(131) 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] -(124) ColumnarToRow [codegen id : 1] +(132) ColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#117] +Subquery:17 Hosting operator id = 76 Hosting Expression = ReusedSubquery Subquery scalar-subquery#72, [id=#73] + +Subquery:18 Hosting operator id = 75 Hosting Expression = ReusedSubquery Subquery scalar-subquery#72, [id=#73] + 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 cd4817202..1210748c5 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,180 +1,173 @@ 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 (24) + WholeStageCodegen (4) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [sales] Subquery #4 WholeStageCodegen (2) HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - InputAdapter - Exchange #13 + ColumnarToRow + InputAdapter + CometColumnarExchange #14 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [quantity,list_price] [sum,count,sum,count] + ColumnarToRow + InputAdapter + CometUnion [quantity,list_price] + CometProject [ss_quantity,ss_list_price] [quantity,list_price] + CometBroadcastHashJoin [ss_quantity,ss_list_price,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_quantity,cs_list_price,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_quantity,ws_list_price,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] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #1 + RowToColumnar WholeStageCodegen (1) - HashAggregate [quantity,list_price] [sum,count,sum,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] ColumnarToRow InputAdapter - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,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_quantity,cs_list_price,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_quantity,ws_list_price,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 (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] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (4) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - ColumnarToRow - InputAdapter + CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #2 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + Subquery #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometBroadcastExchange [ss_item_sk] #3 + CometProject [i_item_sk] [ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,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] - InputAdapter - BroadcastExchange #4 - 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 (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,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 - CometBroadcastHashJoin [i_item_sk,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 [i_brand_id,i_class_id,i_category_id] #8 - CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [cs_item_sk,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - 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 [d_date_sk] #10 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #10 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ws_item_sk,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - ReusedExchange [d_date_sk] #10 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - BroadcastHashJoin [i_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter + CometBroadcastExchange [brand_id,class_id,category_id] #4 + CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + CometHashAggregate [brand_id,class_id,category_id] + CometColumnarExchange [brand_id,class_id,category_id] #5 + 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,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 + CometBroadcastHashJoin [i_item_sk,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 [i_brand_id,i_class_id,i_category_id] #8 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + 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 [d_date_sk] #10 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #10 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #11 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + ReusedExchange [d_date_sk] #10 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] 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 - ReusedExchange [ss_item_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [ss_item_sk] #3 + CometBroadcastExchange [d_date_sk] #13 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #2 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #2 InputAdapter - BroadcastExchange #14 - WholeStageCodegen (23) + BroadcastExchange #15 + WholeStageCodegen (3) 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] #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] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_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 #16 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #6 - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - InputAdapter - ReusedExchange [ss_item_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - InputAdapter - ReusedExchange [d_date_sk] #16 + ColumnarToRow + InputAdapter + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #16 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + ColumnarToRow + InputAdapter + CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_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 #17 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #6 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + Subquery #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + ReusedExchange [ss_item_sk] #3 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + CometBroadcastExchange [d_date_sk] #18 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #6 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #6 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 a22a52dc6..9015f7e90 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,27 +1,29 @@ == Physical Plan == -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) +TakeOrderedAndProject (25) ++- * HashAggregate (24) + +- * ColumnarToRow (23) + +- CometColumnarExchange (22) + +- RowToColumnar (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 @@ -122,50 +124,56 @@ Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#2))] Aggregate Attributes [1]: [sum#13] Results [2]: [ca_zip#9, sum#14] -(21) Exchange +(21) RowToColumnar Input [2]: [ca_zip#9, sum#14] -Arguments: hashpartitioning(ca_zip#9, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(22) HashAggregate [codegen id : 2] +(22) CometColumnarExchange +Input [2]: [ca_zip#9, sum#14] +Arguments: hashpartitioning(ca_zip#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(23) ColumnarToRow [codegen id : 2] +Input [2]: [ca_zip#9, sum#14] + +(24) 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))#15] Results [2]: [ca_zip#9, MakeDecimal(sum(UnscaledValue(cs_sales_price#2))#15,17,2) AS sum(cs_sales_price)#16] -(23) TakeOrderedAndProject +(25) 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 (28) -+- * ColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan parquet spark_catalog.default.date_dim (24) +BroadcastExchange (30) ++- * ColumnarToRow (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan parquet spark_catalog.default.date_dim (26) -(24) Scan parquet spark_catalog.default.date_dim +(26) 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 -(25) CometFilter +(27) 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)) -(26) CometProject +(28) CometProject Input [3]: [d_date_sk#10, d_year#11, d_qoy#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(27) ColumnarToRow [codegen id : 1] +(29) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(28) BroadcastExchange +(30) BroadcastExchange Input [1]: [d_date_sk#10] 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 a34cd9c1b..f466e8189 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,35 +1,37 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] WholeStageCodegen (2) HashAggregate [ca_zip,sum] [sum(UnscaledValue(cs_sales_price)),sum(cs_sales_price),sum] - InputAdapter - Exchange [ca_zip] #1 - WholeStageCodegen (1) - HashAggregate [ca_zip,cs_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [cs_sales_price,ca_zip] - CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,ca_zip,d_date_sk] - CometProject [cs_sales_price,cs_sold_date_sk,ca_zip] - CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_state,ca_zip] - CometProject [cs_sales_price,cs_sold_date_sk,c_current_addr_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk,c_customer_sk,c_current_addr_sk] - CometFilter [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_state,ca_zip] #4 - CometFilter [ca_address_sk,ca_state,ca_zip] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + ColumnarToRow + InputAdapter + CometColumnarExchange [ca_zip] #1 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [ca_zip,cs_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [cs_sales_price,ca_zip] + CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,ca_zip,d_date_sk] + CometProject [cs_sales_price,cs_sold_date_sk,ca_zip] + CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_state,ca_zip] + CometProject [cs_sales_price,cs_sold_date_sk,c_current_addr_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk,c_customer_sk,c_current_addr_sk] + CometFilter [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange [ca_address_sk,ca_state,ca_zip] #4 + CometFilter [ca_address_sk,ca_state,ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + 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/q16/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/explain.txt index ccec341ad..ebff95d06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/explain.txt @@ -1,49 +1,51 @@ == Physical Plan == -* HashAggregate (45) -+- Exchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * SortMergeJoin LeftAnti (19) - : : : :- * Project (13) - : : : : +- * SortMergeJoin LeftSemi (12) - : : : : :- * Sort (6) - : : : : : +- Exchange (5) - : : : : : +- * ColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : +- * Sort (11) - : : : : +- Exchange (10) - : : : : +- * ColumnarToRow (9) - : : : : +- CometProject (8) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (7) - : : : +- * Sort (18) - : : : +- Exchange (17) - : : : +- * ColumnarToRow (16) - : : : +- CometProject (15) - : : : +- CometScan parquet spark_catalog.default.catalog_returns (14) - : : +- BroadcastExchange (24) - : : +- * ColumnarToRow (23) - : : +- CometProject (22) - : : +- CometFilter (21) - : : +- CometScan parquet spark_catalog.default.date_dim (20) - : +- 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.call_center (34) +* HashAggregate (47) ++- * ColumnarToRow (46) + +- CometColumnarExchange (45) + +- RowToColumnar (44) + +- * HashAggregate (43) + +- * HashAggregate (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * SortMergeJoin LeftAnti (19) + : : : :- * Project (13) + : : : : +- * SortMergeJoin LeftSemi (12) + : : : : :- * ColumnarToRow (6) + : : : : : +- CometSort (5) + : : : : : +- CometColumnarExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : +- * ColumnarToRow (11) + : : : : +- CometSort (10) + : : : : +- CometColumnarExchange (9) + : : : : +- CometProject (8) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (7) + : : : +- * ColumnarToRow (18) + : : : +- CometSort (17) + : : : +- CometColumnarExchange (16) + : : : +- CometProject (15) + : : : +- CometScan parquet spark_catalog.default.catalog_returns (14) + : : +- BroadcastExchange (24) + : : +- * ColumnarToRow (23) + : : +- CometProject (22) + : : +- CometFilter (21) + : : +- CometScan parquet spark_catalog.default.date_dim (20) + : +- 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.call_center (34) (1) Scan parquet spark_catalog.default.catalog_sales @@ -61,16 +63,16 @@ Condition : ((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -(4) ColumnarToRow [codegen id : 1] +(4) CometColumnarExchange Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] +Arguments: hashpartitioning(cs_order_number#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(5) Exchange +(5) CometSort Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: hashpartitioning(cs_order_number#5, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_order_number#5 ASC NULLS FIRST] -(6) Sort [codegen id : 2] +(6) ColumnarToRow [codegen id : 1] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: [cs_order_number#5 ASC NULLS FIRST], false, 0 (7) Scan parquet spark_catalog.default.catalog_sales Output [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11] @@ -82,24 +84,24 @@ ReadSchema: struct Input [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11] Arguments: [cs_warehouse_sk#9, cs_order_number#10], [cs_warehouse_sk#9, cs_order_number#10] -(9) ColumnarToRow [codegen id : 3] +(9) CometColumnarExchange Input [2]: [cs_warehouse_sk#9, cs_order_number#10] +Arguments: hashpartitioning(cs_order_number#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(10) Exchange +(10) CometSort Input [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: hashpartitioning(cs_order_number#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: [cs_warehouse_sk#9, cs_order_number#10], [cs_order_number#10 ASC NULLS FIRST] -(11) Sort [codegen id : 4] +(11) ColumnarToRow [codegen id : 2] Input [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: [cs_order_number#10 ASC NULLS FIRST], false, 0 -(12) SortMergeJoin [codegen id : 5] +(12) SortMergeJoin [codegen id : 3] Left keys [1]: [cs_order_number#5] Right keys [1]: [cs_order_number#10] Join type: LeftSemi Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#9) -(13) Project [codegen id : 5] +(13) Project [codegen id : 3] Output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] @@ -113,18 +115,18 @@ ReadSchema: struct Input [2]: [cr_order_number#12, cr_returned_date_sk#13] Arguments: [cr_order_number#12], [cr_order_number#12] -(16) ColumnarToRow [codegen id : 6] +(16) CometColumnarExchange Input [1]: [cr_order_number#12] +Arguments: hashpartitioning(cr_order_number#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(17) Exchange +(17) CometSort Input [1]: [cr_order_number#12] -Arguments: hashpartitioning(cr_order_number#12, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: [cr_order_number#12], [cr_order_number#12 ASC NULLS FIRST] -(18) Sort [codegen id : 7] +(18) ColumnarToRow [codegen id : 4] Input [1]: [cr_order_number#12] -Arguments: [cr_order_number#12 ASC NULLS FIRST], false, 0 -(19) SortMergeJoin [codegen id : 11] +(19) SortMergeJoin [codegen id : 8] Left keys [1]: [cs_order_number#5] Right keys [1]: [cr_order_number#12] Join type: LeftAnti @@ -145,20 +147,20 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2002-02-01)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(23) ColumnarToRow [codegen id : 8] +(23) ColumnarToRow [codegen id : 5] Input [1]: [d_date_sk#14] (24) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(25) BroadcastHashJoin [codegen id : 11] +(25) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_ship_date_sk#1] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(26) Project [codegen id : 11] +(26) Project [codegen id : 8] Output [5]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#14] @@ -177,20 +179,20 @@ Condition : ((isnotnull(ca_state#17) AND (ca_state#17 = GA)) AND isnotnull(ca_ad Input [2]: [ca_address_sk#16, ca_state#17] Arguments: [ca_address_sk#16], [ca_address_sk#16] -(30) ColumnarToRow [codegen id : 9] +(30) ColumnarToRow [codegen id : 6] Input [1]: [ca_address_sk#16] (31) BroadcastExchange Input [1]: [ca_address_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(32) BroadcastHashJoin [codegen id : 11] +(32) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_ship_addr_sk#2] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(33) Project [codegen id : 11] +(33) Project [codegen id : 8] Output [4]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [6]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#16] @@ -209,49 +211,55 @@ Condition : ((isnotnull(cc_county#19) AND (cc_county#19 = Williamson County)) AN Input [2]: [cc_call_center_sk#18, cc_county#19] Arguments: [cc_call_center_sk#18], [cc_call_center_sk#18] -(37) ColumnarToRow [codegen id : 10] +(37) ColumnarToRow [codegen id : 7] Input [1]: [cc_call_center_sk#18] (38) BroadcastExchange Input [1]: [cc_call_center_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(39) BroadcastHashJoin [codegen id : 11] +(39) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_call_center_sk#3] Right keys [1]: [cc_call_center_sk#18] Join type: Inner Join condition: None -(40) Project [codegen id : 11] +(40) Project [codegen id : 8] Output [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [5]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#18] -(41) HashAggregate [codegen id : 11] +(41) HashAggregate [codegen id : 8] Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Keys [1]: [cs_order_number#5] Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#20, sum(UnscaledValue(cs_net_profit#7))#21] Results [3]: [cs_order_number#5, sum#22, sum#23] -(42) HashAggregate [codegen id : 11] +(42) HashAggregate [codegen id : 8] Input [3]: [cs_order_number#5, sum#22, sum#23] Keys [1]: [cs_order_number#5] Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#20, sum(UnscaledValue(cs_net_profit#7))#21] Results [3]: [cs_order_number#5, sum#22, sum#23] -(43) HashAggregate [codegen id : 11] +(43) HashAggregate [codegen id : 8] Input [3]: [cs_order_number#5, sum#22, sum#23] Keys: [] Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#20, sum(UnscaledValue(cs_net_profit#7))#21, count(cs_order_number#5)#24] Results [3]: [sum#22, sum#23, count#25] -(44) Exchange +(44) RowToColumnar Input [3]: [sum#22, sum#23, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(45) HashAggregate [codegen id : 12] +(45) CometColumnarExchange +Input [3]: [sum#22, sum#23, count#25] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(46) ColumnarToRow [codegen id : 9] +Input [3]: [sum#22, sum#23, count#25] + +(47) HashAggregate [codegen id : 9] Input [3]: [sum#22, sum#23, count#25] Keys: [] Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/simplified.txt index 8935abb54..605ce3703 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/simplified.txt @@ -1,74 +1,70 @@ -WholeStageCodegen (12) +WholeStageCodegen (9) HashAggregate [sum,sum,count] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] - InputAdapter - Exchange #1 - WholeStageCodegen (11) - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] - HashAggregate [cs_order_number,cs_ext_ship_cost,cs_net_profit] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] - Project [cs_order_number,cs_ext_ship_cost,cs_net_profit] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - BroadcastHashJoin [cs_ship_addr_sk,ca_address_sk] - Project [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk] - SortMergeJoin [cs_order_number,cr_order_number] - InputAdapter - WholeStageCodegen (5) - Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - SortMergeJoin [cs_order_number,cs_order_number,cs_warehouse_sk,cs_warehouse_sk] - InputAdapter - WholeStageCodegen (2) - Sort [cs_order_number] - InputAdapter - Exchange [cs_order_number] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometColumnarExchange #1 + RowToColumnar + WholeStageCodegen (8) + HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] + HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] + HashAggregate [cs_order_number,cs_ext_ship_cost,cs_net_profit] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] + Project [cs_order_number,cs_ext_ship_cost,cs_net_profit] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + BroadcastHashJoin [cs_ship_addr_sk,ca_address_sk] + Project [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk] + SortMergeJoin [cs_order_number,cr_order_number] + InputAdapter + WholeStageCodegen (3) + Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + SortMergeJoin [cs_order_number,cs_order_number,cs_warehouse_sk,cs_warehouse_sk] + InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometSort [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + CometColumnarExchange [cs_order_number] #2 CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] - InputAdapter - WholeStageCodegen (4) - Sort [cs_order_number] - InputAdapter - Exchange [cs_order_number] #3 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter + InputAdapter + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometSort [cs_warehouse_sk,cs_order_number] + CometColumnarExchange [cs_order_number] #3 CometProject [cs_warehouse_sk,cs_order_number] CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] - InputAdapter - WholeStageCodegen (7) - Sort [cr_order_number] - InputAdapter - Exchange [cr_order_number] #4 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter + InputAdapter + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometSort [cr_order_number] + CometColumnarExchange [cr_order_number] #4 CometProject [cr_order_number] CometScan parquet spark_catalog.default.catalog_returns [cr_order_number,cr_returned_date_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (8) + BroadcastExchange #6 + WholeStageCodegen (6) ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) + BroadcastExchange #7 + WholeStageCodegen (7) ColumnarToRow InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometProject [cc_call_center_sk] - CometFilter [cc_call_center_sk,cc_county] - CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_county] + CometProject [cc_call_center_sk] + CometFilter [cc_call_center_sk,cc_county] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_county] 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 d3739ed75..2da3a5a21 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,46 +1,48 @@ == Physical Plan == -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) +TakeOrderedAndProject (44) ++- * HashAggregate (43) + +- * ColumnarToRow (42) + +- CometColumnarExchange (41) + +- RowToColumnar (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 @@ -231,80 +233,86 @@ Functions [9]: [partial_count(ss_quantity#5), partial_avg(ss_quantity#5), partia 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] -(40) Exchange +(40) RowToColumnar 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] -(41) HashAggregate [codegen id : 2] +(41) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(42) ColumnarToRow [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] + +(43) 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)#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] -(42) TakeOrderedAndProject +(44) 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 (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 +(45) 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 -(44) CometFilter +(46) 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)) -(45) CometProject +(47) CometProject Input [2]: [d_date_sk#19, d_quarter_name#20] Arguments: [d_date_sk#19], [d_date_sk#19] -(46) ColumnarToRow [codegen id : 1] +(48) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] -(47) BroadcastExchange +(49) BroadcastExchange Input [1]: [d_date_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] 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) +BroadcastExchange (54) ++- * ColumnarToRow (53) + +- CometProject (52) + +- CometFilter (51) + +- CometScan parquet spark_catalog.default.date_dim (50) -(48) Scan parquet spark_catalog.default.date_dim +(50) 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 -(49) CometFilter +(51) 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)) -(50) CometProject +(52) CometProject Input [2]: [d_date_sk#21, d_quarter_name#22] Arguments: [d_date_sk#21], [d_date_sk#21] -(51) ColumnarToRow [codegen id : 1] +(53) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#21] -(52) BroadcastExchange +(54) BroadcastExchange Input [1]: [d_date_sk#21] 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/q17/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/simplified.txt index 4a634b069..0604dcd57 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,63 +1,65 @@ 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 (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 (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] - ColumnarToRow - InputAdapter - CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_sk,i_item_id,i_item_desc] - CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_state] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,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 [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_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_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_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 [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,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 - CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - 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_date_sk,d_quarter_name] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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 [d_date_sk] #6 + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id,i_item_desc,s_state] #1 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_sk,i_item_id,i_item_desc] + CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_state] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,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 [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_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_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_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 [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,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_quarter_name] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + 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_date_sk,d_quarter_name] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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 [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_quarter_name] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [s_store_sk,s_state] #8 - CometFilter [s_store_sk,s_state] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange [s_store_sk,s_state] #8 + CometFilter [s_store_sk,s_state] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 + CometFilter [i_item_sk,i_item_id,i_item_desc] + 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 acbc8e654..49d68974b 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,45 +1,47 @@ == Physical Plan == -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) +TakeOrderedAndProject (43) ++- * HashAggregate (42) + +- * ColumnarToRow (41) + +- CometColumnarExchange (40) + +- RowToColumnar (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 @@ -224,50 +226,56 @@ Functions [7]: [partial_avg(cast(cs_quantity#4 as decimal(12,2))), partial_avg(c 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] -(39) Exchange +(39) RowToColumnar 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] -(40) HashAggregate [codegen id : 2] +(40) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(41) ColumnarToRow [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] + +(42) 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)))#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] -(41) TakeOrderedAndProject +(43) 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 (46) -+- * ColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan parquet spark_catalog.default.date_dim (42) +BroadcastExchange (48) ++- * ColumnarToRow (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan parquet spark_catalog.default.date_dim (44) -(42) Scan parquet spark_catalog.default.date_dim +(44) 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 -(43) CometFilter +(45) 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)) -(44) CometProject +(46) CometProject Input [2]: [d_date_sk#25, d_year#26] Arguments: [d_date_sk#25], [d_date_sk#25] -(45) ColumnarToRow [codegen id : 1] +(47) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] -(46) BroadcastExchange +(48) BroadcastExchange Input [1]: [d_date_sk#25] 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 c49bfc6c8..660161251 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,53 +1,55 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] 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 (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] - 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,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,i_item_sk,i_item_id] - 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_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,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 [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,ca_address_sk,ca_county,ca_state,ca_country] - 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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - 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_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,cd_demo_sk,cd_dep_count] - CometFilter [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] - 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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange [cd_demo_sk] #5 - CometFilter [cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #8 - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] #1 + RowToColumnar + 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] + 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,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,i_item_sk,i_item_id] + 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_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,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 [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,ca_address_sk,ca_county,ca_state,ca_country] + 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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + 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_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,cd_demo_sk,cd_dep_count] + CometFilter [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] + 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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 + CometProject [cd_demo_sk,cd_dep_count] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometBroadcastExchange [cd_demo_sk] #5 + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] + CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] + CometBroadcastExchange [d_date_sk] #7 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_item_id] #8 + CometFilter [i_item_sk,i_item_id] + 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 0353b1c2f..270facf74 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,38 +1,40 @@ == Physical Plan == -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) +TakeOrderedAndProject (36) ++- * HashAggregate (35) + +- * ColumnarToRow (34) + +- CometColumnarExchange (33) + +- RowToColumnar (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 @@ -185,18 +187,24 @@ 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] -(32) Exchange +(32) RowToColumnar 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=1] -(33) HashAggregate [codegen id : 2] +(33) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(34) ColumnarToRow [codegen id : 2] +Input [5]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, sum#22] + +(35) 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] -(34) TakeOrderedAndProject +(36) 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 60caecf91..082614b7f 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,38 +1,40 @@ TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] 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 (1) - HashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,ss_ext_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip,s_store_sk,s_zip] - CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk,ca_address_sk,ca_zip] - 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,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_customer_sk,c_current_addr_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,ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_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 [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] #3 - CometProject [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #4 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_zip] #5 - CometFilter [ca_address_sk,ca_zip] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange [s_store_sk,s_zip] #6 - CometFilter [s_store_sk,s_zip] - CometScan parquet spark_catalog.default.store [s_store_sk,s_zip] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_brand,i_brand_id,i_manufact_id,i_manufact] #1 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,ss_ext_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip,s_store_sk,s_zip] + CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk,ca_address_sk,ca_zip] + 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,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_customer_sk,c_current_addr_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,ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] + CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_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 [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] #3 + CometProject [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #4 + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange [ca_address_sk,ca_zip] #5 + CometFilter [ca_address_sk,ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] + CometBroadcastExchange [s_store_sk,s_zip] #6 + CometFilter [s_store_sk,s_zip] + 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 48b744d96..70af0c5f4 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,39 +1,44 @@ == Physical Plan == -* 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) +* ColumnarToRow (40) ++- CometSort (39) + +- CometColumnarExchange (38) + +- RowToColumnar (37) + +- * Project (36) + +- * BroadcastHashJoin Inner BuildRight (35) + :- * Project (23) + : +- * BroadcastHashJoin Inner BuildRight (22) + : :- * HashAggregate (16) + : : +- * ColumnarToRow (15) + : : +- CometColumnarExchange (14) + : : +- RowToColumnar (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 (21) + : +- * ColumnarToRow (20) + : +- CometProject (19) + : +- CometFilter (18) + : +- CometScan parquet spark_catalog.default.date_dim (17) + +- BroadcastExchange (34) + +- * Project (33) + +- * BroadcastHashJoin Inner BuildRight (32) + :- * HashAggregate (26) + : +- * ColumnarToRow (25) + : +- ReusedExchange (24) + +- BroadcastExchange (31) + +- * ColumnarToRow (30) + +- CometProject (29) + +- CometFilter (28) + +- CometScan parquet spark_catalog.default.date_dim (27) (1) Scan parquet spark_catalog.default.web_sales @@ -96,110 +101,125 @@ Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) 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] -(13) Exchange +(13) RowToColumnar 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=1] -(14) HashAggregate [codegen id : 6] +(14) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(15) ColumnarToRow [codegen id : 6] +Input [8]: [d_week_seq#10, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25] + +(16) 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] -(15) Scan parquet spark_catalog.default.date_dim +(17) 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 -(16) CometFilter +(18) 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)) -(17) CometProject +(19) CometProject Input [2]: [d_week_seq#40, d_year#41] Arguments: [d_week_seq#40], [d_week_seq#40] -(18) ColumnarToRow [codegen id : 2] +(20) ColumnarToRow [codegen id : 2] Input [1]: [d_week_seq#40] -(19) BroadcastExchange +(21) BroadcastExchange Input [1]: [d_week_seq#40] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(20) BroadcastHashJoin [codegen id : 6] +(22) BroadcastHashJoin [codegen id : 6] Left keys [1]: [d_week_seq#10] Right keys [1]: [d_week_seq#40] Join type: Inner Join condition: None -(21) Project [codegen id : 6] +(23) 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] -(22) ReusedExchange [Reuses operator id: 13] +(24) ReusedExchange [Reuses operator id: 14] Output [8]: [d_week_seq#50, sum#51, sum#52, sum#53, sum#54, sum#55, sum#56, sum#57] -(23) HashAggregate [codegen id : 5] +(25) ColumnarToRow [codegen id : 5] +Input [8]: [d_week_seq#50, sum#51, sum#52, sum#53, sum#54, sum#55, sum#56, sum#57] + +(26) 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] -(24) Scan parquet spark_catalog.default.date_dim +(27) 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 -(25) CometFilter +(28) 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)) -(26) CometProject +(29) CometProject Input [2]: [d_week_seq#67, d_year#68] Arguments: [d_week_seq#67], [d_week_seq#67] -(27) ColumnarToRow [codegen id : 4] +(30) ColumnarToRow [codegen id : 4] Input [1]: [d_week_seq#67] -(28) BroadcastExchange +(31) BroadcastExchange Input [1]: [d_week_seq#67] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(29) BroadcastHashJoin [codegen id : 5] +(32) BroadcastHashJoin [codegen id : 5] Left keys [1]: [d_week_seq#50] Right keys [1]: [d_week_seq#67] Join type: Inner Join condition: None -(30) Project [codegen id : 5] +(33) 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] -(31) BroadcastExchange +(34) 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=4] -(32) BroadcastHashJoin [codegen id : 6] +(35) 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 -(33) Project [codegen id : 6] +(36) 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] -(34) Exchange +(37) RowToColumnar +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] + +(38) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=5] + +(39) CometSort 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=5] +Arguments: [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], [d_week_seq1#42 ASC NULLS FIRST] -(35) Sort [codegen id : 7] +(40) ColumnarToRow [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 535c61622..a4f4107c6 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,50 +1,55 @@ WholeStageCodegen (7) - Sort [d_week_seq1] + ColumnarToRow InputAdapter - Exchange [d_week_seq1] #1 - 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] - 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] - InputAdapter - Exchange [d_week_seq] #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] + CometSort [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] + CometColumnarExchange [d_week_seq1] #1 + RowToColumnar + 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] + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [d_week_seq] #2 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometProject [sales_price,d_week_seq,d_day_name] + CometBroadcastHashJoin [sold_date_sk,sales_price,d_date_sk,d_week_seq,d_day_name] + CometUnion [sold_date_sk,sales_price] + 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] + CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 + CometFilter [d_date_sk,d_week_seq,d_day_name] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometProject [sales_price,d_week_seq,d_day_name] - CometBroadcastHashJoin [sold_date_sk,sales_price,d_date_sk,d_week_seq,d_day_name] - CometUnion [sold_date_sk,sales_price] - 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] - CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 - CometFilter [d_date_sk,d_week_seq,d_day_name] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year] - CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] - InputAdapter - BroadcastExchange #5 - 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] - InputAdapter - ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) + BroadcastExchange #5 + 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] ColumnarToRow InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year] - CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] + ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] 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 4bb7f5ca5..b88b1983b 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,26 +1,30 @@ == Physical Plan == -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) +TakeOrderedAndProject (26) ++- * Project (25) + +- Window (24) + +- * ColumnarToRow (23) + +- CometSort (22) + +- CometColumnarExchange (21) + +- RowToColumnar (20) + +- * HashAggregate (19) + +- * ColumnarToRow (18) + +- CometColumnarExchange (17) + +- RowToColumnar (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 @@ -97,66 +101,78 @@ Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] 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] -(16) Exchange +(16) RowToColumnar 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] -(17) HashAggregate [codegen id : 2] +(17) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(18) ColumnarToRow [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] + +(19) 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))#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] -(18) Exchange +(20) RowToColumnar +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17, i_item_id#6] + +(21) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] + +(22) CometSort 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] +Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17, i_item_id#6], [i_class#9 ASC NULLS FIRST] -(19) Sort [codegen id : 3] +(23) ColumnarToRow [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 -(20) Window +(24) 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] -(21) Project [codegen id : 4] +(25) 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] -(22) TakeOrderedAndProject +(26) 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 (27) -+- * ColumnarToRow (26) - +- CometProject (25) - +- CometFilter (24) - +- CometScan parquet spark_catalog.default.date_dim (23) +BroadcastExchange (31) ++- * ColumnarToRow (30) + +- CometProject (29) + +- CometFilter (28) + +- CometScan parquet spark_catalog.default.date_dim (27) -(23) Scan parquet spark_catalog.default.date_dim +(27) 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 +(28) 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 +(29) CometProject Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(26) ColumnarToRow [codegen id : 1] +(30) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(27) BroadcastExchange +(31) BroadcastExchange Input [1]: [d_date_sk#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/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt index 5bfd9a907..9eaa25219 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 @@ -4,35 +4,39 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter Window [_w0,i_class] WholeStageCodegen (3) - Sort [i_class] + ColumnarToRow InputAdapter - Exchange [i_class] #1 - 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 (1) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometColumnarExchange [i_class] #1 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + 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 b2f81dbc1..5b422574b 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 @@ -1,28 +1,30 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * Filter (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.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) +TakeOrderedAndProject (26) ++- * Filter (25) + +- * HashAggregate (24) + +- * ColumnarToRow (23) + +- CometColumnarExchange (22) + +- RowToColumnar (21) + +- * HashAggregate (20) + +- * 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 @@ -123,49 +125,55 @@ Functions [2]: [partial_sum(CASE WHEN (d_date#12 < 2000-03-11) THEN inv_quantity Aggregate Attributes [2]: [sum#13, sum#14] Results [4]: [w_warehouse_name#7, i_item_id#9, sum#15, sum#16] -(21) Exchange +(21) RowToColumnar 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=1] -(22) HashAggregate [codegen id : 2] +(22) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(23) ColumnarToRow [codegen id : 2] +Input [4]: [w_warehouse_name#7, i_item_id#9, sum#15, sum#16] + +(24) 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 : 2] +(25) 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) -(24) TakeOrderedAndProject +(26) 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] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (28) -+- * ColumnarToRow (27) - +- CometFilter (26) - +- CometScan parquet spark_catalog.default.date_dim (25) +BroadcastExchange (30) ++- * ColumnarToRow (29) + +- CometFilter (28) + +- CometScan parquet spark_catalog.default.date_dim (27) -(25) Scan parquet spark_catalog.default.date_dim +(27) 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 -(26) CometFilter +(28) 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)) -(27) ColumnarToRow [codegen id : 1] +(29) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#11, d_date#12] -(28) BroadcastExchange +(30) BroadcastExchange Input [2]: [d_date_sk#11, d_date#12] 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 2c8892b68..7747dd616 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 @@ -2,34 +2,36 @@ TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] 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 (1) - HashAggregate [w_warehouse_name,i_item_id,d_date,inv_quantity_on_hand] [sum,sum,sum,sum] - ColumnarToRow - InputAdapter - CometProject [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] - CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id,d_date_sk,d_date] - CometProject [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id] - CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_sk,i_item_id] - CometProject [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_sk,w_warehouse_name] - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #3 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [i_item_sk,i_item_id] #4 - CometProject [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id,i_current_price] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - CometBroadcastExchange [d_date_sk,d_date] #5 - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + ColumnarToRow + InputAdapter + CometColumnarExchange [w_warehouse_name,i_item_id] #1 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [w_warehouse_name,i_item_id,d_date,inv_quantity_on_hand] [sum,sum,sum,sum] + ColumnarToRow + InputAdapter + CometProject [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id,d_date_sk,d_date] + CometProject [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_sk,i_item_id] + CometProject [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #3 + CometFilter [w_warehouse_sk,w_warehouse_name] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange [i_item_sk,i_item_id] #4 + CometProject [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id,i_current_price] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] + CometBroadcastExchange [d_date_sk,d_date] #5 + CometFilter [d_date_sk,d_date] + 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 342926b93..8e3bf2731 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,28 +1,30 @@ == Physical Plan == -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) +TakeOrderedAndProject (26) ++- * HashAggregate (25) + +- * ColumnarToRow (24) + +- CometColumnarExchange (23) + +- RowToColumnar (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 @@ -127,50 +129,56 @@ Functions [1]: [partial_avg(inv_quantity_on_hand#3)] 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] -(22) Exchange +(22) RowToColumnar 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] -(23) HashAggregate [codegen id : 2] +(23) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(24) ColumnarToRow [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] + +(25) 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)#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] -(24) TakeOrderedAndProject +(26) 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 (29) -+- * ColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan parquet spark_catalog.default.date_dim (25) +BroadcastExchange (31) ++- * ColumnarToRow (30) + +- CometProject (29) + +- CometFilter (28) + +- CometScan parquet spark_catalog.default.date_dim (27) -(25) Scan parquet spark_catalog.default.date_dim +(27) 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 -(26) CometFilter +(28) 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)) -(27) CometProject +(29) CometProject Input [2]: [d_date_sk#6, d_month_seq#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(28) ColumnarToRow [codegen id : 1] +(30) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(29) BroadcastExchange +(31) BroadcastExchange Input [1]: [d_date_sk#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/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/simplified.txt index eb25d6d3e..2bb7b519c 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,36 +1,38 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] 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 (1) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] - 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,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] - CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] - CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange [w_warehouse_sk] #5 - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] + 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,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] + CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] + CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometBroadcastExchange [w_warehouse_sk] #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 10f78aaa1..a849066c6 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,71 +1,75 @@ == Physical Plan == -* 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 (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) +* HashAggregate (71) ++- * ColumnarToRow (70) + +- CometColumnarExchange (69) + +- RowToColumnar (68) + +- * HashAggregate (67) + +- Union (66) + :- * Project (47) + : +- * BroadcastHashJoin Inner BuildRight (46) + : :- * Project (44) + : : +- * SortMergeJoin LeftSemi (43) + : : :- * ColumnarToRow (25) + : : : +- CometSort (24) + : : : +- CometColumnarExchange (23) + : : : +- CometProject (22) + : : : +- CometBroadcastHashJoin (21) + : : : :- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : +- CometBroadcastExchange (20) + : : : +- CometProject (19) + : : : +- CometFilter (18) + : : : +- CometHashAggregate (17) + : : : +- CometColumnarExchange (16) + : : : +- CometHashAggregate (15) + : : : +- CometProject (14) + : : : +- CometBroadcastHashJoin (13) + : : : :- CometProject (9) + : : : : +- CometBroadcastHashJoin (8) + : : : : :- CometFilter (3) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (2) + : : : : +- CometBroadcastExchange (7) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.date_dim (4) + : : : +- CometBroadcastExchange (12) + : : : +- CometFilter (11) + : : : +- CometScan parquet spark_catalog.default.item (10) + : : +- * Sort (42) + : : +- * Project (41) + : : +- * Filter (40) + : : +- * HashAggregate (39) + : : +- * ColumnarToRow (38) + : : +- CometColumnarExchange (37) + : : +- RowToColumnar (36) + : : +- * HashAggregate (35) + : : +- * ColumnarToRow (34) + : : +- CometProject (33) + : : +- CometBroadcastHashJoin (32) + : : :- CometProject (28) + : : : +- CometFilter (27) + : : : +- CometScan parquet spark_catalog.default.store_sales (26) + : : +- CometBroadcastExchange (31) + : : +- CometFilter (30) + : : +- CometScan parquet spark_catalog.default.customer (29) + : +- ReusedExchange (45) + +- * Project (65) + +- * BroadcastHashJoin Inner BuildRight (64) + :- * Project (62) + : +- * SortMergeJoin LeftSemi (61) + : :- * ColumnarToRow (54) + : : +- CometSort (53) + : : +- CometColumnarExchange (52) + : : +- CometProject (51) + : : +- CometBroadcastHashJoin (50) + : : :- CometScan parquet spark_catalog.default.web_sales (48) + : : +- ReusedExchange (49) + : +- * Sort (60) + : +- * Project (59) + : +- * Filter (58) + : +- * HashAggregate (57) + : +- * ColumnarToRow (56) + : +- ReusedExchange (55) + +- ReusedExchange (63) (1) Scan parquet spark_catalog.default.catalog_sales @@ -75,10 +79,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 : 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 +(2) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_item_sk#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] @@ -86,508 +87,535 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(4) CometFilter +(3) CometFilter Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] Condition : isnotnull(ss_item_sk#7) -(5) Scan parquet spark_catalog.default.date_dim +(4) 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) CometFilter +(5) 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) CometProject +(6) 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 +(7) CometBroadcastExchange Input [2]: [d_date_sk#10, d_date#11] Arguments: [d_date_sk#10, d_date#11] -(9) CometBroadcastHashJoin +(8) 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 +(9) 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] -(11) Scan parquet spark_catalog.default.item +(10) 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 -(12) CometFilter +(11) CometFilter Input [2]: [i_item_sk#13, i_item_desc#14] Condition : isnotnull(i_item_sk#13) -(13) CometBroadcastExchange +(12) CometBroadcastExchange Input [2]: [i_item_sk#13, i_item_desc#14] Arguments: [i_item_sk#13, i_item_desc#14] -(14) CometBroadcastHashJoin +(13) 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) CometProject +(14) 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) CometHashAggregate +(15) 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)] -(17) ColumnarToRow [codegen id : 1] -Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] - -(18) Exchange +(16) CometColumnarExchange 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] +Arguments: hashpartitioning(_groupingexpression#15, i_item_sk#13, d_date#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(19) HashAggregate [codegen id : 2] +(17) CometHashAggregate 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#13 AS item_sk#18, count(1)#17 AS cnt#19] -(20) Filter [codegen id : 2] -Input [2]: [item_sk#18, cnt#19] -Condition : (cnt#19 > 4) +(18) CometFilter +Input [2]: [item_sk#17, cnt#18] +Condition : (cnt#18 > 4) -(21) Project [codegen id : 2] -Output [1]: [item_sk#18] -Input [2]: [item_sk#18, cnt#19] +(19) CometProject +Input [2]: [item_sk#17, cnt#18] +Arguments: [item_sk#17], [item_sk#17] -(22) BroadcastExchange -Input [1]: [item_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(20) CometBroadcastExchange +Input [1]: [item_sk#17] +Arguments: [item_sk#17] -(23) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#2] -Right keys [1]: [item_sk#18] -Join type: LeftSemi -Join condition: None +(21) CometBroadcastHashJoin +Left output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Right output [1]: [item_sk#17] +Arguments: [cs_item_sk#2], [item_sk#17], LeftSemi, BuildRight -(24) Project [codegen id : 3] -Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +(22) CometProject Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(25) Exchange +(23) CometColumnarExchange 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=3] +Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(26) Sort [codegen id : 4] +(24) CometSort 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 +Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1 ASC NULLS FIRST] -(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] +(25) ColumnarToRow [codegen id : 1] +Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] + +(26) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(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) CometFilter +Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] +Condition : isnotnull(ss_customer_sk#19) -(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) CometProject +Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] +Arguments: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21], [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] -(30) Scan parquet spark_catalog.default.customer -Output [1]: [c_customer_sk#24] +(29) Scan parquet spark_catalog.default.customer +Output [1]: [c_customer_sk#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(31) CometFilter -Input [1]: [c_customer_sk#24] -Condition : isnotnull(c_customer_sk#24) - -(32) CometBroadcastExchange -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24] - -(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) 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] - -(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] - -(37) Exchange -Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] -Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(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] - -(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]))) - -(40) Project [codegen id : 6] -Output [1]: [c_customer_sk#24] -Input [2]: [c_customer_sk#24, ssales#30] - -(41) Sort [codegen id : 6] -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 - -(42) SortMergeJoin [codegen id : 8] +(30) CometFilter +Input [1]: [c_customer_sk#23] +Condition : isnotnull(c_customer_sk#23) + +(31) CometBroadcastExchange +Input [1]: [c_customer_sk#23] +Arguments: [c_customer_sk#23] + +(32) CometBroadcastHashJoin +Left output [3]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] +Right output [1]: [c_customer_sk#23] +Arguments: [ss_customer_sk#19], [c_customer_sk#23], Inner, BuildRight + +(33) CometProject +Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, c_customer_sk#23] +Arguments: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23], [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] + +(34) ColumnarToRow [codegen id : 2] +Input [3]: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] + +(35) HashAggregate [codegen id : 2] +Input [3]: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] +Keys [1]: [c_customer_sk#23] +Functions [1]: [partial_sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] +Aggregate Attributes [2]: [sum#24, isEmpty#25] +Results [3]: [c_customer_sk#23, sum#26, isEmpty#27] + +(36) RowToColumnar +Input [3]: [c_customer_sk#23, sum#26, isEmpty#27] + +(37) CometColumnarExchange +Input [3]: [c_customer_sk#23, sum#26, isEmpty#27] +Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(38) ColumnarToRow [codegen id : 3] +Input [3]: [c_customer_sk#23, sum#26, isEmpty#27] + +(39) HashAggregate [codegen id : 3] +Input [3]: [c_customer_sk#23, sum#26, isEmpty#27] +Keys [1]: [c_customer_sk#23] +Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] +Aggregate Attributes [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))#28] +Results [2]: [c_customer_sk#23, sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))#28 AS ssales#29] + +(40) Filter [codegen id : 3] +Input [2]: [c_customer_sk#23, ssales#29] +Condition : (isnotnull(ssales#29) AND (cast(ssales#29 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#30, [id=#31]))) + +(41) Project [codegen id : 3] +Output [1]: [c_customer_sk#23] +Input [2]: [c_customer_sk#23, ssales#29] + +(42) Sort [codegen id : 3] +Input [1]: [c_customer_sk#23] +Arguments: [c_customer_sk#23 ASC NULLS FIRST], false, 0 + +(43) SortMergeJoin [codegen id : 5] Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#24] +Right keys [1]: [c_customer_sk#23] Join type: LeftSemi Join condition: None -(43) Project [codegen id : 8] +(44) Project [codegen id : 5] 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] -(44) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#33] +(45) ReusedExchange [Reuses operator id: 76] +Output [1]: [d_date_sk#32] -(45) BroadcastHashJoin [codegen id : 8] +(46) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#33] +Right keys [1]: [d_date_sk#32] Join type: Inner Join condition: None -(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] +(47) Project [codegen id : 5] +Output [1]: [(cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4) AS sales#33] +Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#32] -(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] +(48) Scan parquet spark_catalog.default.web_sales +Output [5]: [ws_item_sk#34, ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] 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#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#39)] ReadSchema: struct -(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] +(49) ReusedExchange [Reuses operator id: 20] +Output [1]: [item_sk#40] -(49) ReusedExchange [Reuses operator id: 22] -Output [1]: [item_sk#41] +(50) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#34, ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] +Right output [1]: [item_sk#40] +Arguments: [ws_item_sk#34], [item_sk#40], LeftSemi, BuildRight -(50) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ws_item_sk#35] -Right keys [1]: [item_sk#41] -Join type: LeftSemi -Join condition: None +(51) CometProject +Input [5]: [ws_item_sk#34, ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] +Arguments: [ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38], [ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] + +(52) CometColumnarExchange +Input [4]: [ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] +Arguments: hashpartitioning(ws_bill_customer_sk#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(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] +(53) CometSort +Input [4]: [ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] +Arguments: [ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38], [ws_bill_customer_sk#35 ASC NULLS FIRST] -(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=5] +(54) ColumnarToRow [codegen id : 6] +Input [4]: [ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] -(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 +(55) ReusedExchange [Reuses operator id: 37] +Output [3]: [c_customer_sk#41, sum#42, isEmpty#43] -(54) ReusedExchange [Reuses operator id: 37] -Output [3]: [c_customer_sk#42, sum#43, isEmpty#44] +(56) ColumnarToRow [codegen id : 8] +Input [3]: [c_customer_sk#41, sum#42, isEmpty#43] -(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] +(57) HashAggregate [codegen id : 8] +Input [3]: [c_customer_sk#41, sum#42, isEmpty#43] +Keys [1]: [c_customer_sk#41] +Functions [1]: [sum((cast(ss_quantity#44 as decimal(10,0)) * ss_sales_price#45))] +Aggregate Attributes [1]: [sum((cast(ss_quantity#44 as decimal(10,0)) * ss_sales_price#45))#28] +Results [2]: [c_customer_sk#41, sum((cast(ss_quantity#44 as decimal(10,0)) * ss_sales_price#45))#28 AS ssales#46] -(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]))) +(58) Filter [codegen id : 8] +Input [2]: [c_customer_sk#41, ssales#46] +Condition : (isnotnull(ssales#46) AND (cast(ssales#46 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#30, [id=#31]))) -(57) Project [codegen id : 14] -Output [1]: [c_customer_sk#42] -Input [2]: [c_customer_sk#42, ssales#47] +(59) Project [codegen id : 8] +Output [1]: [c_customer_sk#41] +Input [2]: [c_customer_sk#41, ssales#46] -(58) Sort [codegen id : 14] -Input [1]: [c_customer_sk#42] -Arguments: [c_customer_sk#42 ASC NULLS FIRST], false, 0 +(60) Sort [codegen id : 8] +Input [1]: [c_customer_sk#41] +Arguments: [c_customer_sk#41 ASC NULLS FIRST], false, 0 -(59) SortMergeJoin [codegen id : 16] -Left keys [1]: [ws_bill_customer_sk#36] -Right keys [1]: [c_customer_sk#42] +(61) SortMergeJoin [codegen id : 10] +Left keys [1]: [ws_bill_customer_sk#35] +Right keys [1]: [c_customer_sk#41] Join type: LeftSemi Join condition: None -(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] +(62) Project [codegen id : 10] +Output [3]: [ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] +Input [4]: [ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] -(61) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#48] +(63) ReusedExchange [Reuses operator id: 76] +Output [1]: [d_date_sk#47] -(62) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ws_sold_date_sk#39] -Right keys [1]: [d_date_sk#48] +(64) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ws_sold_date_sk#38] +Right keys [1]: [d_date_sk#47] Join type: Inner Join condition: None -(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] +(65) Project [codegen id : 10] +Output [1]: [(cast(ws_quantity#36 as decimal(10,0)) * ws_list_price#37) AS sales#48] +Input [4]: [ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38, d_date_sk#47] -(64) Union +(66) Union -(65) HashAggregate [codegen id : 17] -Input [1]: [sales#34] +(67) HashAggregate [codegen id : 11] +Input [1]: [sales#33] Keys: [] -Functions [1]: [partial_sum(sales#34)] -Aggregate Attributes [2]: [sum#50, isEmpty#51] -Results [2]: [sum#52, isEmpty#53] +Functions [1]: [partial_sum(sales#33)] +Aggregate Attributes [2]: [sum#49, isEmpty#50] +Results [2]: [sum#51, isEmpty#52] -(66) Exchange -Input [2]: [sum#52, isEmpty#53] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +(68) RowToColumnar +Input [2]: [sum#51, isEmpty#52] -(67) HashAggregate [codegen id : 18] -Input [2]: [sum#52, isEmpty#53] +(69) CometColumnarExchange +Input [2]: [sum#51, isEmpty#52] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(70) ColumnarToRow [codegen id : 12] +Input [2]: [sum#51, isEmpty#52] + +(71) HashAggregate [codegen id : 12] +Input [2]: [sum#51, isEmpty#52] Keys: [] -Functions [1]: [sum(sales#34)] -Aggregate Attributes [1]: [sum(sales#34)#54] -Results [1]: [sum(sales#34)#54 AS sum(sales)#55] +Functions [1]: [sum(sales#33)] +Aggregate Attributes [1]: [sum(sales#33)#53] +Results [1]: [sum(sales#33)#53 AS sum(sales)#54] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (72) -+- * ColumnarToRow (71) - +- CometProject (70) - +- CometFilter (69) - +- CometScan parquet spark_catalog.default.date_dim (68) +BroadcastExchange (76) ++- * ColumnarToRow (75) + +- CometProject (74) + +- CometFilter (73) + +- CometScan parquet spark_catalog.default.date_dim (72) -(68) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#33, d_year#56, d_moy#57] +(72) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#32, d_year#55, d_moy#56] 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 -(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)) +(73) CometFilter +Input [3]: [d_date_sk#32, d_year#55, d_moy#56] +Condition : ((((isnotnull(d_year#55) AND isnotnull(d_moy#56)) AND (d_year#55 = 2000)) AND (d_moy#56 = 2)) AND isnotnull(d_date_sk#32)) -(70) CometProject -Input [3]: [d_date_sk#33, d_year#56, d_moy#57] -Arguments: [d_date_sk#33], [d_date_sk#33] +(74) CometProject +Input [3]: [d_date_sk#32, d_year#55, d_moy#56] +Arguments: [d_date_sk#32], [d_date_sk#32] -(71) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#33] +(75) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#32] -(72) BroadcastExchange -Input [1]: [d_date_sk#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +(76) BroadcastExchange +Input [1]: [d_date_sk#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (77) -+- * ColumnarToRow (76) - +- CometProject (75) - +- CometFilter (74) - +- CometScan parquet spark_catalog.default.date_dim (73) +Subquery:2 Hosting operator id = 2 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +BroadcastExchange (81) ++- * ColumnarToRow (80) + +- CometProject (79) + +- CometFilter (78) + +- CometScan parquet spark_catalog.default.date_dim (77) -(73) Scan parquet spark_catalog.default.date_dim +(77) 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 -(74) CometFilter +(78) 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)) -(75) CometProject +(79) 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] -(76) ColumnarToRow [codegen id : 1] +(80) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_date#11] -(77) BroadcastExchange +(81) BroadcastExchange Input [2]: [d_date_sk#10, d_date#11] -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] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] + +Subquery:3 Hosting operator id = 40 Hosting Expression = Subquery scalar-subquery#30, [id=#31] +* HashAggregate (103) ++- * ColumnarToRow (102) + +- CometColumnarExchange (101) + +- RowToColumnar (100) + +- * HashAggregate (99) + +- * HashAggregate (98) + +- * ColumnarToRow (97) + +- CometColumnarExchange (96) + +- RowToColumnar (95) + +- * HashAggregate (94) + +- * ColumnarToRow (93) + +- CometProject (92) + +- CometBroadcastHashJoin (91) + :- CometProject (86) + : +- CometBroadcastHashJoin (85) + : :- CometFilter (83) + : : +- CometScan parquet spark_catalog.default.store_sales (82) + : +- ReusedExchange (84) + +- CometBroadcastExchange (90) + +- CometProject (89) + +- CometFilter (88) + +- CometScan parquet spark_catalog.default.date_dim (87) + + +(82) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_customer_sk#57, ss_quantity#58, ss_sales_price#59, ss_sold_date_sk#60] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#61), dynamicpruningexpression(ss_sold_date_sk#61 IN dynamicpruning#62)] +PartitionFilters: [isnotnull(ss_sold_date_sk#60), dynamicpruningexpression(ss_sold_date_sk#60 IN dynamicpruning#61)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(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) +(83) CometFilter +Input [4]: [ss_customer_sk#57, ss_quantity#58, ss_sales_price#59, ss_sold_date_sk#60] +Condition : isnotnull(ss_customer_sk#57) -(80) ReusedExchange [Reuses operator id: 32] -Output [1]: [c_customer_sk#63] +(84) ReusedExchange [Reuses operator id: 31] +Output [1]: [c_customer_sk#62] -(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 +(85) CometBroadcastHashJoin +Left output [4]: [ss_customer_sk#57, ss_quantity#58, ss_sales_price#59, ss_sold_date_sk#60] +Right output [1]: [c_customer_sk#62] +Arguments: [ss_customer_sk#57], [c_customer_sk#62], 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] +(86) CometProject +Input [5]: [ss_customer_sk#57, ss_quantity#58, ss_sales_price#59, ss_sold_date_sk#60, c_customer_sk#62] +Arguments: [ss_quantity#58, ss_sales_price#59, ss_sold_date_sk#60, c_customer_sk#62], [ss_quantity#58, ss_sales_price#59, ss_sold_date_sk#60, c_customer_sk#62] -(83) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#64, d_year#65] +(87) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#63, d_year#64] 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) 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)) - -(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] - -(89) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#59, ss_sales_price#60, c_customer_sk#63] - -(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#63, sum#68, isEmpty#69] - -(91) Exchange -Input [3]: [c_customer_sk#63, sum#68, isEmpty#69] -Arguments: hashpartitioning(c_customer_sk#63, 5), ENSURE_REQUIREMENTS, [plan_id=9] - -(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] - -(93) HashAggregate [codegen id : 2] -Input [1]: [csales#71] +(88) CometFilter +Input [2]: [d_date_sk#63, d_year#64] +Condition : (d_year#64 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#63)) + +(89) CometProject +Input [2]: [d_date_sk#63, d_year#64] +Arguments: [d_date_sk#63], [d_date_sk#63] + +(90) CometBroadcastExchange +Input [1]: [d_date_sk#63] +Arguments: [d_date_sk#63] + +(91) CometBroadcastHashJoin +Left output [4]: [ss_quantity#58, ss_sales_price#59, ss_sold_date_sk#60, c_customer_sk#62] +Right output [1]: [d_date_sk#63] +Arguments: [ss_sold_date_sk#60], [d_date_sk#63], Inner, BuildRight + +(92) CometProject +Input [5]: [ss_quantity#58, ss_sales_price#59, ss_sold_date_sk#60, c_customer_sk#62, d_date_sk#63] +Arguments: [ss_quantity#58, ss_sales_price#59, c_customer_sk#62], [ss_quantity#58, ss_sales_price#59, c_customer_sk#62] + +(93) ColumnarToRow [codegen id : 1] +Input [3]: [ss_quantity#58, ss_sales_price#59, c_customer_sk#62] + +(94) HashAggregate [codegen id : 1] +Input [3]: [ss_quantity#58, ss_sales_price#59, c_customer_sk#62] +Keys [1]: [c_customer_sk#62] +Functions [1]: [partial_sum((cast(ss_quantity#58 as decimal(10,0)) * ss_sales_price#59))] +Aggregate Attributes [2]: [sum#65, isEmpty#66] +Results [3]: [c_customer_sk#62, sum#67, isEmpty#68] + +(95) RowToColumnar +Input [3]: [c_customer_sk#62, sum#67, isEmpty#68] + +(96) CometColumnarExchange +Input [3]: [c_customer_sk#62, sum#67, isEmpty#68] +Arguments: hashpartitioning(c_customer_sk#62, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(97) ColumnarToRow [codegen id : 2] +Input [3]: [c_customer_sk#62, sum#67, isEmpty#68] + +(98) HashAggregate [codegen id : 2] +Input [3]: [c_customer_sk#62, sum#67, isEmpty#68] +Keys [1]: [c_customer_sk#62] +Functions [1]: [sum((cast(ss_quantity#58 as decimal(10,0)) * ss_sales_price#59))] +Aggregate Attributes [1]: [sum((cast(ss_quantity#58 as decimal(10,0)) * ss_sales_price#59))#69] +Results [1]: [sum((cast(ss_quantity#58 as decimal(10,0)) * ss_sales_price#59))#69 AS csales#70] + +(99) HashAggregate [codegen id : 2] +Input [1]: [csales#70] Keys: [] -Functions [1]: [partial_max(csales#71)] -Aggregate Attributes [1]: [max#72] -Results [1]: [max#73] +Functions [1]: [partial_max(csales#70)] +Aggregate Attributes [1]: [max#71] +Results [1]: [max#72] + +(100) RowToColumnar +Input [1]: [max#72] + +(101) CometColumnarExchange +Input [1]: [max#72] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(94) Exchange -Input [1]: [max#73] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] +(102) ColumnarToRow [codegen id : 3] +Input [1]: [max#72] -(95) HashAggregate [codegen id : 3] -Input [1]: [max#73] +(103) HashAggregate [codegen id : 3] +Input [1]: [max#72] Keys: [] -Functions [1]: [max(csales#71)] -Aggregate Attributes [1]: [max(csales#71)#74] -Results [1]: [max(csales#71)#74 AS tpcds_cmax#75] +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 = 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) +Subquery:4 Hosting operator id = 82 Hosting Expression = ss_sold_date_sk#60 IN dynamicpruning#61 +BroadcastExchange (108) ++- * ColumnarToRow (107) + +- CometProject (106) + +- CometFilter (105) + +- CometScan parquet spark_catalog.default.date_dim (104) -(96) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#64, d_year#65] +(104) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#63, d_year#64] 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 -(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)) +(105) CometFilter +Input [2]: [d_date_sk#63, d_year#64] +Condition : (d_year#64 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#63)) -(98) CometProject -Input [2]: [d_date_sk#64, d_year#65] -Arguments: [d_date_sk#64], [d_date_sk#64] +(106) CometProject +Input [2]: [d_date_sk#63, d_year#64] +Arguments: [d_date_sk#63], [d_date_sk#63] -(99) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#64] +(107) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#63] -(100) BroadcastExchange -Input [1]: [d_date_sk#64] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] +(108) BroadcastExchange +Input [1]: [d_date_sk#63] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] -Subquery:5 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#39 IN dynamicpruning#6 +Subquery:5 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#6 -Subquery:6 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] +Subquery:6 Hosting operator id = 58 Hosting Expression = ReusedSubquery Subquery scalar-subquery#30, [id=#31] 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 412f27ffb..317d50c28 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,26 +1,26 @@ -WholeStageCodegen (18) +WholeStageCodegen (12) HashAggregate [sum,isEmpty] [sum(sales),sum(sales),sum,isEmpty] - InputAdapter - Exchange #1 - WholeStageCodegen (17) - HashAggregate [sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - 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 (4) - Sort [cs_bill_customer_sk] - InputAdapter - Exchange [cs_bill_customer_sk] #2 - WholeStageCodegen (3) - Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - BroadcastHashJoin [cs_item_sk,item_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometColumnarExchange #1 + RowToColumnar + WholeStageCodegen (11) + HashAggregate [sales] [sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (5) + 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 (1) + ColumnarToRow + InputAdapter + CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + CometColumnarExchange [cs_bill_customer_sk] #2 + CometProject [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,121 +30,119 @@ WholeStageCodegen (18) CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] - CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] - CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [i_item_sk,i_item_desc] #8 - CometFilter [i_item_sk,i_item_desc] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - InputAdapter - WholeStageCodegen (6) - Sort [c_customer_sk] - Project [c_customer_sk] - Filter [ssales] - Subquery #3 - WholeStageCodegen (3) - HashAggregate [max] [max(csales),tpcds_cmax,max] - InputAdapter - 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] #12 - WholeStageCodegen (1) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - ColumnarToRow - InputAdapter - CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] - CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_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 #13 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - 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] #9 - WholeStageCodegen (5) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + CometBroadcastExchange [item_sk] #4 + CometProject [item_sk] + CometFilter [item_sk,cnt] + CometHashAggregate [item_sk,cnt,_groupingexpression,i_item_sk,d_date,count,count(1)] + CometColumnarExchange [_groupingexpression,i_item_sk,d_date] #5 + CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] + CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] + CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] + CometProject [ss_item_sk,d_date] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + CometBroadcastExchange [d_date_sk,d_date] #7 + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + CometBroadcastExchange [i_item_sk,i_item_desc] #8 + CometFilter [i_item_sk,i_item_desc] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + InputAdapter + WholeStageCodegen (3) + Sort [c_customer_sk] + Project [c_customer_sk] + Filter [ssales] + Subquery #3 + WholeStageCodegen (3) + HashAggregate [max] [max(csales),tpcds_cmax,max] ColumnarToRow InputAdapter - CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] - CometProject [ss_customer_sk,ss_quantity,ss_sales_price] - CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [c_customer_sk] #10 - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk] - InputAdapter - ReusedExchange [d_date_sk] #3 - 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 (12) - Sort [ws_bill_customer_sk] - InputAdapter - 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] + CometColumnarExchange #11 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_sk] #12 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometProject [ss_quantity,ss_sales_price,c_customer_sk] + CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] + CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_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 #13 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [c_customer_sk] #10 + CometBroadcastExchange [d_date_sk] #14 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + 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] ColumnarToRow InputAdapter + CometColumnarExchange [c_customer_sk] #9 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometProject [ss_quantity,ss_sales_price,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] + CometProject [ss_customer_sk,ss_quantity,ss_sales_price] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometBroadcastExchange [c_customer_sk] #10 + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk] + InputAdapter + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (10) + 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 (6) + ColumnarToRow + InputAdapter + CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + CometColumnarExchange [ws_bill_customer_sk] #15 + CometProject [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [item_sk] #4 + ReusedExchange [item_sk] #4 + InputAdapter + WholeStageCodegen (8) + 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] + ColumnarToRow + InputAdapter + ReusedExchange [c_customer_sk,sum,isEmpty] #9 InputAdapter - 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] #9 - InputAdapter - ReusedExchange [d_date_sk] #3 + 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 2e6ff8aaa..876b496c5 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,92 +1,99 @@ == Physical Plan == -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 (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) +TakeOrderedAndProject (95) ++- Union (94) + :- * HashAggregate (67) + : +- * ColumnarToRow (66) + : +- CometColumnarExchange (65) + : +- RowToColumnar (64) + : +- * HashAggregate (63) + : +- * Project (62) + : +- * BroadcastHashJoin Inner BuildRight (61) + : :- * Project (59) + : : +- * BroadcastHashJoin Inner BuildRight (58) + : : :- * SortMergeJoin LeftSemi (44) + : : : :- * ColumnarToRow (26) + : : : : +- CometSort (25) + : : : : +- CometColumnarExchange (24) + : : : : +- CometProject (23) + : : : : +- CometBroadcastHashJoin (22) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : +- CometBroadcastExchange (21) + : : : : +- CometProject (20) + : : : : +- CometFilter (19) + : : : : +- CometHashAggregate (18) + : : : : +- CometColumnarExchange (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 (43) + : : : +- * Project (42) + : : : +- * Filter (41) + : : : +- * HashAggregate (40) + : : : +- * ColumnarToRow (39) + : : : +- CometColumnarExchange (38) + : : : +- RowToColumnar (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) + : : +- BroadcastExchange (57) + : : +- * SortMergeJoin LeftSemi (56) + : : :- * ColumnarToRow (49) + : : : +- CometSort (48) + : : : +- CometColumnarExchange (47) + : : : +- CometFilter (46) + : : : +- CometScan parquet spark_catalog.default.customer (45) + : : +- * Sort (55) + : : +- * Project (54) + : : +- * Filter (53) + : : +- * HashAggregate (52) + : : +- * ColumnarToRow (51) + : : +- ReusedExchange (50) + : +- ReusedExchange (60) + +- * HashAggregate (93) + +- * ColumnarToRow (92) + +- CometColumnarExchange (91) + +- RowToColumnar (90) + +- * HashAggregate (89) + +- * Project (88) + +- * BroadcastHashJoin Inner BuildRight (87) + :- * Project (85) + : +- * BroadcastHashJoin Inner BuildRight (84) + : :- * SortMergeJoin LeftSemi (82) + : : :- * ColumnarToRow (75) + : : : +- CometSort (74) + : : : +- CometColumnarExchange (73) + : : : +- CometProject (72) + : : : +- CometBroadcastHashJoin (71) + : : : :- CometFilter (69) + : : : : +- CometScan parquet spark_catalog.default.web_sales (68) + : : : +- ReusedExchange (70) + : : +- * Sort (81) + : : +- * Project (80) + : : +- * Filter (79) + : : +- * HashAggregate (78) + : : +- * ColumnarToRow (77) + : : +- ReusedExchange (76) + : +- ReusedExchange (83) + +- ReusedExchange (86) (1) Scan parquet spark_catalog.default.catalog_sales @@ -101,10 +108,7 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] Condition : isnotnull(ss_item_sk#7) -(6) Scan parquet spark_catalog.default.date_dim +(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 -(7) CometFilter +(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)) -(8) CometProject +(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] -(9) CometBroadcastExchange +(8) CometBroadcastExchange Input [2]: [d_date_sk#10, d_date#11] Arguments: [d_date_sk#10, d_date#11] -(10) CometBroadcastHashJoin +(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 -(11) CometProject +(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] -(12) Scan parquet spark_catalog.default.item +(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 -(13) CometFilter +(12) CometFilter Input [2]: [i_item_sk#13, i_item_desc#14] Condition : isnotnull(i_item_sk#13) -(14) CometBroadcastExchange +(13) CometBroadcastExchange Input [2]: [i_item_sk#13, i_item_desc#14] Arguments: [i_item_sk#13, i_item_desc#14] -(15) CometBroadcastHashJoin +(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 -(16) CometProject +(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] -(17) CometHashAggregate +(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)] -(18) ColumnarToRow [codegen id : 1] -Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] - -(19) Exchange +(17) CometColumnarExchange 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] +Arguments: hashpartitioning(_groupingexpression#15, i_item_sk#13, d_date#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(20) HashAggregate [codegen id : 2] +(18) CometHashAggregate 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#13 AS item_sk#18, count(1)#17 AS cnt#19] -(21) Filter [codegen id : 2] -Input [2]: [item_sk#18, cnt#19] -Condition : (cnt#19 > 4) +(19) CometFilter +Input [2]: [item_sk#17, cnt#18] +Condition : (cnt#18 > 4) -(22) Project [codegen id : 2] -Output [1]: [item_sk#18] -Input [2]: [item_sk#18, cnt#19] +(20) CometProject +Input [2]: [item_sk#17, cnt#18] +Arguments: [item_sk#17], [item_sk#17] -(23) BroadcastExchange -Input [1]: [item_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(21) CometBroadcastExchange +Input [1]: [item_sk#17] +Arguments: [item_sk#17] -(24) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#2] -Right keys [1]: [item_sk#18] -Join type: LeftSemi -Join condition: None +(22) CometBroadcastHashJoin +Left output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Right output [1]: [item_sk#17] +Arguments: [cs_item_sk#2], [item_sk#17], LeftSemi, BuildRight -(25) Project [codegen id : 3] -Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +(23) CometProject Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(26) Exchange +(24) CometColumnarExchange 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=3] +Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(27) Sort [codegen id : 4] +(25) CometSort 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 +Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1 ASC NULLS FIRST] -(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] +(26) ColumnarToRow [codegen id : 1] +Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] + +(27) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(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) CometFilter +Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] +Condition : isnotnull(ss_customer_sk#19) -(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) CometProject +Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] +Arguments: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21], [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] -(31) Scan parquet spark_catalog.default.customer -Output [1]: [c_customer_sk#24] +(30) Scan parquet spark_catalog.default.customer +Output [1]: [c_customer_sk#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(32) CometFilter -Input [1]: [c_customer_sk#24] -Condition : isnotnull(c_customer_sk#24) - -(33) CometBroadcastExchange -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24] - -(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) 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] - -(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] - -(38) Exchange -Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] -Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(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] - -(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]))) - -(41) Project [codegen id : 6] -Output [1]: [c_customer_sk#24] -Input [2]: [c_customer_sk#24, ssales#30] - -(42) Sort [codegen id : 6] -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 - -(43) SortMergeJoin [codegen id : 13] +(31) CometFilter +Input [1]: [c_customer_sk#23] +Condition : isnotnull(c_customer_sk#23) + +(32) CometBroadcastExchange +Input [1]: [c_customer_sk#23] +Arguments: [c_customer_sk#23] + +(33) CometBroadcastHashJoin +Left output [3]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] +Right output [1]: [c_customer_sk#23] +Arguments: [ss_customer_sk#19], [c_customer_sk#23], Inner, BuildRight + +(34) CometProject +Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, c_customer_sk#23] +Arguments: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23], [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] + +(35) ColumnarToRow [codegen id : 2] +Input [3]: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] + +(36) HashAggregate [codegen id : 2] +Input [3]: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] +Keys [1]: [c_customer_sk#23] +Functions [1]: [partial_sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] +Aggregate Attributes [2]: [sum#24, isEmpty#25] +Results [3]: [c_customer_sk#23, sum#26, isEmpty#27] + +(37) RowToColumnar +Input [3]: [c_customer_sk#23, sum#26, isEmpty#27] + +(38) CometColumnarExchange +Input [3]: [c_customer_sk#23, sum#26, isEmpty#27] +Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(39) ColumnarToRow [codegen id : 3] +Input [3]: [c_customer_sk#23, sum#26, isEmpty#27] + +(40) HashAggregate [codegen id : 3] +Input [3]: [c_customer_sk#23, sum#26, isEmpty#27] +Keys [1]: [c_customer_sk#23] +Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] +Aggregate Attributes [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))#28] +Results [2]: [c_customer_sk#23, sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))#28 AS ssales#29] + +(41) Filter [codegen id : 3] +Input [2]: [c_customer_sk#23, ssales#29] +Condition : (isnotnull(ssales#29) AND (cast(ssales#29 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#30, [id=#31]))) + +(42) Project [codegen id : 3] +Output [1]: [c_customer_sk#23] +Input [2]: [c_customer_sk#23, ssales#29] + +(43) Sort [codegen id : 3] +Input [1]: [c_customer_sk#23] +Arguments: [c_customer_sk#23 ASC NULLS FIRST], false, 0 + +(44) SortMergeJoin [codegen id : 9] Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#24] +Right keys [1]: [c_customer_sk#23] Join type: LeftSemi Join condition: None -(44) Scan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] +(45) Scan parquet spark_catalog.default.customer +Output [3]: [c_customer_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)] ReadSchema: struct -(45) CometFilter -Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] -Condition : isnotnull(c_customer_sk#33) +(46) CometFilter +Input [3]: [c_customer_sk#32, c_first_name#33, c_last_name#34] +Condition : isnotnull(c_customer_sk#32) -(46) ColumnarToRow [codegen id : 7] -Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] +(47) CometColumnarExchange +Input [3]: [c_customer_sk#32, c_first_name#33, c_last_name#34] +Arguments: hashpartitioning(c_customer_sk#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(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=5] +(48) CometSort +Input [3]: [c_customer_sk#32, c_first_name#33, c_last_name#34] +Arguments: [c_customer_sk#32, c_first_name#33, c_last_name#34], [c_customer_sk#32 ASC NULLS FIRST] -(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 +(49) ColumnarToRow [codegen id : 4] +Input [3]: [c_customer_sk#32, c_first_name#33, c_last_name#34] -(49) ReusedExchange [Reuses operator id: 38] -Output [3]: [c_customer_sk#24, sum#27, isEmpty#28] +(50) ReusedExchange [Reuses operator id: 38] +Output [3]: [c_customer_sk#23, sum#26, isEmpty#27] -(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] +(51) ColumnarToRow [codegen id : 6] +Input [3]: [c_customer_sk#23, sum#26, isEmpty#27] -(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]))) +(52) HashAggregate [codegen id : 6] +Input [3]: [c_customer_sk#23, sum#26, isEmpty#27] +Keys [1]: [c_customer_sk#23] +Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] +Aggregate Attributes [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))#28] +Results [2]: [c_customer_sk#23, sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))#28 AS ssales#29] -(52) Project [codegen id : 10] -Output [1]: [c_customer_sk#24] -Input [2]: [c_customer_sk#24, ssales#30] +(53) Filter [codegen id : 6] +Input [2]: [c_customer_sk#23, ssales#29] +Condition : (isnotnull(ssales#29) AND (cast(ssales#29 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#30, [id=#31]))) -(53) Sort [codegen id : 10] -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 +(54) Project [codegen id : 6] +Output [1]: [c_customer_sk#23] +Input [2]: [c_customer_sk#23, ssales#29] -(54) SortMergeJoin [codegen id : 11] -Left keys [1]: [c_customer_sk#33] -Right keys [1]: [c_customer_sk#24] +(55) Sort [codegen id : 6] +Input [1]: [c_customer_sk#23] +Arguments: [c_customer_sk#23 ASC NULLS FIRST], false, 0 + +(56) SortMergeJoin [codegen id : 7] +Left keys [1]: [c_customer_sk#32] +Right keys [1]: [c_customer_sk#23] Join type: LeftSemi Join condition: None -(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=6] +(57) BroadcastExchange +Input [3]: [c_customer_sk#32, c_first_name#33, c_last_name#34] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(56) BroadcastHashJoin [codegen id : 13] +(58) BroadcastHashJoin [codegen id : 9] Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#33] +Right keys [1]: [c_customer_sk#32] Join type: Inner Join condition: None -(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] +(59) Project [codegen id : 9] +Output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#33, c_last_name#34] +Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#32, c_first_name#33, c_last_name#34] -(58) ReusedExchange [Reuses operator id: 93] -Output [1]: [d_date_sk#36] +(60) ReusedExchange [Reuses operator id: 100] +Output [1]: [d_date_sk#35] -(59) BroadcastHashJoin [codegen id : 13] +(61) BroadcastHashJoin [codegen id : 9] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#36] +Right keys [1]: [d_date_sk#35] Join type: Inner Join condition: None -(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] +(62) Project [codegen id : 9] +Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#33, c_last_name#34] +Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#33, c_last_name#34, d_date_sk#35] -(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] +(63) HashAggregate [codegen id : 9] +Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#33, c_last_name#34] +Keys [2]: [c_last_name#34, c_first_name#33] 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#36, isEmpty#37] +Results [4]: [c_last_name#34, c_first_name#33, sum#38, isEmpty#39] + +(64) RowToColumnar +Input [4]: [c_last_name#34, c_first_name#33, sum#38, isEmpty#39] + +(65) CometColumnarExchange +Input [4]: [c_last_name#34, c_first_name#33, sum#38, isEmpty#39] +Arguments: hashpartitioning(c_last_name#34, c_first_name#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(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=7] +(66) ColumnarToRow [codegen id : 10] +Input [4]: [c_last_name#34, c_first_name#33, sum#38, isEmpty#39] -(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] +(67) HashAggregate [codegen id : 10] +Input [4]: [c_last_name#34, c_first_name#33, sum#38, isEmpty#39] +Keys [2]: [c_last_name#34, c_first_name#33] 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))#40] +Results [3]: [c_last_name#34, c_first_name#33, sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))#40 AS sales#41] -(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] +(68) Scan parquet spark_catalog.default.web_sales +Output [5]: [ws_item_sk#42, ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45, ws_sold_date_sk#46] 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#46), dynamicpruningexpression(ws_sold_date_sk#46 IN dynamicpruning#47)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(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) +(69) CometFilter +Input [5]: [ws_item_sk#42, ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45, ws_sold_date_sk#46] +Condition : isnotnull(ws_bill_customer_sk#43) -(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] +(70) ReusedExchange [Reuses operator id: 21] +Output [1]: [item_sk#48] -(67) ReusedExchange [Reuses operator id: 23] -Output [1]: [item_sk#49] +(71) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#42, ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45, ws_sold_date_sk#46] +Right output [1]: [item_sk#48] +Arguments: [ws_item_sk#42], [item_sk#48], LeftSemi, BuildRight -(68) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#43] -Right keys [1]: [item_sk#49] -Join type: LeftSemi -Join condition: None +(72) CometProject +Input [5]: [ws_item_sk#42, ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45, ws_sold_date_sk#46] +Arguments: [ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45, ws_sold_date_sk#46], [ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45, ws_sold_date_sk#46] + +(73) CometColumnarExchange +Input [4]: [ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45, ws_sold_date_sk#46] +Arguments: hashpartitioning(ws_bill_customer_sk#43, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(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] +(74) CometSort +Input [4]: [ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45, ws_sold_date_sk#46] +Arguments: [ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45, ws_sold_date_sk#46], [ws_bill_customer_sk#43 ASC NULLS FIRST] -(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=8] +(75) ColumnarToRow [codegen id : 11] +Input [4]: [ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45, ws_sold_date_sk#46] -(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 +(76) ReusedExchange [Reuses operator id: 38] +Output [3]: [c_customer_sk#49, sum#50, isEmpty#51] -(72) ReusedExchange [Reuses operator id: 38] -Output [3]: [c_customer_sk#50, sum#51, isEmpty#52] +(77) ColumnarToRow [codegen id : 13] +Input [3]: [c_customer_sk#49, sum#50, isEmpty#51] -(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] +(78) HashAggregate [codegen id : 13] +Input [3]: [c_customer_sk#49, sum#50, isEmpty#51] +Keys [1]: [c_customer_sk#49] +Functions [1]: [sum((cast(ss_quantity#52 as decimal(10,0)) * ss_sales_price#53))] +Aggregate Attributes [1]: [sum((cast(ss_quantity#52 as decimal(10,0)) * ss_sales_price#53))#28] +Results [2]: [c_customer_sk#49, sum((cast(ss_quantity#52 as decimal(10,0)) * ss_sales_price#53))#28 AS ssales#54] -(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]))) +(79) Filter [codegen id : 13] +Input [2]: [c_customer_sk#49, ssales#54] +Condition : (isnotnull(ssales#54) AND (cast(ssales#54 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#30, [id=#31]))) -(75) Project [codegen id : 20] -Output [1]: [c_customer_sk#50] -Input [2]: [c_customer_sk#50, ssales#55] +(80) Project [codegen id : 13] +Output [1]: [c_customer_sk#49] +Input [2]: [c_customer_sk#49, ssales#54] -(76) Sort [codegen id : 20] -Input [1]: [c_customer_sk#50] -Arguments: [c_customer_sk#50 ASC NULLS FIRST], false, 0 +(81) Sort [codegen id : 13] +Input [1]: [c_customer_sk#49] +Arguments: [c_customer_sk#49 ASC NULLS FIRST], false, 0 -(77) SortMergeJoin [codegen id : 27] -Left keys [1]: [ws_bill_customer_sk#44] -Right keys [1]: [c_customer_sk#50] +(82) SortMergeJoin [codegen id : 19] +Left keys [1]: [ws_bill_customer_sk#43] +Right keys [1]: [c_customer_sk#49] Join type: LeftSemi Join condition: None -(78) ReusedExchange [Reuses operator id: 55] -Output [3]: [c_customer_sk#56, c_first_name#57, c_last_name#58] +(83) ReusedExchange [Reuses operator id: 57] +Output [3]: [c_customer_sk#55, c_first_name#56, c_last_name#57] -(79) BroadcastHashJoin [codegen id : 27] -Left keys [1]: [ws_bill_customer_sk#44] -Right keys [1]: [c_customer_sk#56] +(84) BroadcastHashJoin [codegen id : 19] +Left keys [1]: [ws_bill_customer_sk#43] +Right keys [1]: [c_customer_sk#55] Join type: Inner Join condition: None -(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] +(85) Project [codegen id : 19] +Output [5]: [ws_quantity#44, ws_list_price#45, ws_sold_date_sk#46, c_first_name#56, c_last_name#57] +Input [7]: [ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45, ws_sold_date_sk#46, c_customer_sk#55, c_first_name#56, c_last_name#57] -(81) ReusedExchange [Reuses operator id: 93] -Output [1]: [d_date_sk#59] +(86) ReusedExchange [Reuses operator id: 100] +Output [1]: [d_date_sk#58] -(82) BroadcastHashJoin [codegen id : 27] -Left keys [1]: [ws_sold_date_sk#47] -Right keys [1]: [d_date_sk#59] +(87) BroadcastHashJoin [codegen id : 19] +Left keys [1]: [ws_sold_date_sk#46] +Right keys [1]: [d_date_sk#58] Join type: Inner Join condition: None -(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] +(88) Project [codegen id : 19] +Output [4]: [ws_quantity#44, ws_list_price#45, c_first_name#56, c_last_name#57] +Input [6]: [ws_quantity#44, ws_list_price#45, ws_sold_date_sk#46, c_first_name#56, c_last_name#57, d_date_sk#58] -(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] +(89) HashAggregate [codegen id : 19] +Input [4]: [ws_quantity#44, ws_list_price#45, c_first_name#56, c_last_name#57] +Keys [2]: [c_last_name#57, c_first_name#56] +Functions [1]: [partial_sum((cast(ws_quantity#44 as decimal(10,0)) * ws_list_price#45))] +Aggregate Attributes [2]: [sum#59, isEmpty#60] +Results [4]: [c_last_name#57, c_first_name#56, sum#61, isEmpty#62] -(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=9] +(90) RowToColumnar +Input [4]: [c_last_name#57, c_first_name#56, sum#61, isEmpty#62] -(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] +(91) CometColumnarExchange +Input [4]: [c_last_name#57, c_first_name#56, sum#61, isEmpty#62] +Arguments: hashpartitioning(c_last_name#57, c_first_name#56, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(87) Union +(92) ColumnarToRow [codegen id : 20] +Input [4]: [c_last_name#57, c_first_name#56, sum#61, isEmpty#62] -(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] +(93) HashAggregate [codegen id : 20] +Input [4]: [c_last_name#57, c_first_name#56, sum#61, isEmpty#62] +Keys [2]: [c_last_name#57, c_first_name#56] +Functions [1]: [sum((cast(ws_quantity#44 as decimal(10,0)) * ws_list_price#45))] +Aggregate Attributes [1]: [sum((cast(ws_quantity#44 as decimal(10,0)) * ws_list_price#45))#63] +Results [3]: [c_last_name#57, c_first_name#56, sum((cast(ws_quantity#44 as decimal(10,0)) * ws_list_price#45))#63 AS sales#64] + +(94) Union + +(95) TakeOrderedAndProject +Input [3]: [c_last_name#34, c_first_name#33, sales#41] +Arguments: 100, [c_last_name#34 ASC NULLS FIRST, c_first_name#33 ASC NULLS FIRST, sales#41 ASC NULLS FIRST], [c_last_name#34, c_first_name#33, sales#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (93) -+- * ColumnarToRow (92) - +- CometProject (91) - +- CometFilter (90) - +- CometScan parquet spark_catalog.default.date_dim (89) +BroadcastExchange (100) ++- * ColumnarToRow (99) + +- CometProject (98) + +- CometFilter (97) + +- CometScan parquet spark_catalog.default.date_dim (96) -(89) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#36, d_year#66, d_moy#67] +(96) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#35, d_year#65, d_moy#66] 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 -(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)) +(97) CometFilter +Input [3]: [d_date_sk#35, d_year#65, d_moy#66] +Condition : ((((isnotnull(d_year#65) AND isnotnull(d_moy#66)) AND (d_year#65 = 2000)) AND (d_moy#66 = 2)) AND isnotnull(d_date_sk#35)) -(91) CometProject -Input [3]: [d_date_sk#36, d_year#66, d_moy#67] -Arguments: [d_date_sk#36], [d_date_sk#36] +(98) CometProject +Input [3]: [d_date_sk#35, d_year#65, d_moy#66] +Arguments: [d_date_sk#35], [d_date_sk#35] -(92) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#36] +(99) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#35] -(93) BroadcastExchange -Input [1]: [d_date_sk#36] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] +(100) BroadcastExchange +Input [1]: [d_date_sk#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -Subquery:2 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (98) -+- * ColumnarToRow (97) - +- CometProject (96) - +- CometFilter (95) - +- CometScan parquet spark_catalog.default.date_dim (94) +Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +BroadcastExchange (105) ++- * ColumnarToRow (104) + +- CometProject (103) + +- CometFilter (102) + +- CometScan parquet spark_catalog.default.date_dim (101) -(94) Scan parquet spark_catalog.default.date_dim +(101) 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 -(95) CometFilter +(102) 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)) -(96) CometProject +(103) 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] -(97) ColumnarToRow [codegen id : 1] +(104) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_date#11] -(98) BroadcastExchange +(105) BroadcastExchange Input [2]: [d_date_sk#10, d_date#11] -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] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] + +Subquery:3 Hosting operator id = 41 Hosting Expression = Subquery scalar-subquery#30, [id=#31] +* HashAggregate (127) ++- * ColumnarToRow (126) + +- CometColumnarExchange (125) + +- RowToColumnar (124) + +- * HashAggregate (123) + +- * HashAggregate (122) + +- * ColumnarToRow (121) + +- CometColumnarExchange (120) + +- RowToColumnar (119) + +- * HashAggregate (118) + +- * ColumnarToRow (117) + +- CometProject (116) + +- CometBroadcastHashJoin (115) + :- CometProject (110) + : +- CometBroadcastHashJoin (109) + : :- CometFilter (107) + : : +- CometScan parquet spark_catalog.default.store_sales (106) + : +- ReusedExchange (108) + +- CometBroadcastExchange (114) + +- CometProject (113) + +- CometFilter (112) + +- CometScan parquet spark_catalog.default.date_dim (111) + + +(106) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_customer_sk#67, ss_quantity#68, ss_sales_price#69, ss_sold_date_sk#70] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#71), dynamicpruningexpression(ss_sold_date_sk#71 IN dynamicpruning#72)] +PartitionFilters: [isnotnull(ss_sold_date_sk#70), dynamicpruningexpression(ss_sold_date_sk#70 IN dynamicpruning#71)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(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) +(107) CometFilter +Input [4]: [ss_customer_sk#67, ss_quantity#68, ss_sales_price#69, ss_sold_date_sk#70] +Condition : isnotnull(ss_customer_sk#67) -(101) ReusedExchange [Reuses operator id: 33] -Output [1]: [c_customer_sk#73] +(108) ReusedExchange [Reuses operator id: 32] +Output [1]: [c_customer_sk#72] -(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 +(109) CometBroadcastHashJoin +Left output [4]: [ss_customer_sk#67, ss_quantity#68, ss_sales_price#69, ss_sold_date_sk#70] +Right output [1]: [c_customer_sk#72] +Arguments: [ss_customer_sk#67], [c_customer_sk#72], 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] +(110) CometProject +Input [5]: [ss_customer_sk#67, ss_quantity#68, ss_sales_price#69, ss_sold_date_sk#70, c_customer_sk#72] +Arguments: [ss_quantity#68, ss_sales_price#69, ss_sold_date_sk#70, c_customer_sk#72], [ss_quantity#68, ss_sales_price#69, ss_sold_date_sk#70, c_customer_sk#72] -(104) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#74, d_year#75] +(111) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#73, d_year#74] 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) 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)) - -(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] - -(110) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#69, ss_sales_price#70, c_customer_sk#73] - -(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#73, sum#78, isEmpty#79] - -(112) Exchange -Input [3]: [c_customer_sk#73, sum#78, isEmpty#79] -Arguments: hashpartitioning(c_customer_sk#73, 5), ENSURE_REQUIREMENTS, [plan_id=12] - -(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] - -(114) HashAggregate [codegen id : 2] -Input [1]: [csales#81] +(112) CometFilter +Input [2]: [d_date_sk#73, d_year#74] +Condition : (d_year#74 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#73)) + +(113) CometProject +Input [2]: [d_date_sk#73, d_year#74] +Arguments: [d_date_sk#73], [d_date_sk#73] + +(114) CometBroadcastExchange +Input [1]: [d_date_sk#73] +Arguments: [d_date_sk#73] + +(115) CometBroadcastHashJoin +Left output [4]: [ss_quantity#68, ss_sales_price#69, ss_sold_date_sk#70, c_customer_sk#72] +Right output [1]: [d_date_sk#73] +Arguments: [ss_sold_date_sk#70], [d_date_sk#73], Inner, BuildRight + +(116) CometProject +Input [5]: [ss_quantity#68, ss_sales_price#69, ss_sold_date_sk#70, c_customer_sk#72, d_date_sk#73] +Arguments: [ss_quantity#68, ss_sales_price#69, c_customer_sk#72], [ss_quantity#68, ss_sales_price#69, c_customer_sk#72] + +(117) ColumnarToRow [codegen id : 1] +Input [3]: [ss_quantity#68, ss_sales_price#69, c_customer_sk#72] + +(118) HashAggregate [codegen id : 1] +Input [3]: [ss_quantity#68, ss_sales_price#69, c_customer_sk#72] +Keys [1]: [c_customer_sk#72] +Functions [1]: [partial_sum((cast(ss_quantity#68 as decimal(10,0)) * ss_sales_price#69))] +Aggregate Attributes [2]: [sum#75, isEmpty#76] +Results [3]: [c_customer_sk#72, sum#77, isEmpty#78] + +(119) RowToColumnar +Input [3]: [c_customer_sk#72, sum#77, isEmpty#78] + +(120) CometColumnarExchange +Input [3]: [c_customer_sk#72, sum#77, isEmpty#78] +Arguments: hashpartitioning(c_customer_sk#72, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] + +(121) ColumnarToRow [codegen id : 2] +Input [3]: [c_customer_sk#72, sum#77, isEmpty#78] + +(122) HashAggregate [codegen id : 2] +Input [3]: [c_customer_sk#72, sum#77, isEmpty#78] +Keys [1]: [c_customer_sk#72] +Functions [1]: [sum((cast(ss_quantity#68 as decimal(10,0)) * ss_sales_price#69))] +Aggregate Attributes [1]: [sum((cast(ss_quantity#68 as decimal(10,0)) * ss_sales_price#69))#79] +Results [1]: [sum((cast(ss_quantity#68 as decimal(10,0)) * ss_sales_price#69))#79 AS csales#80] + +(123) HashAggregate [codegen id : 2] +Input [1]: [csales#80] Keys: [] -Functions [1]: [partial_max(csales#81)] -Aggregate Attributes [1]: [max#82] -Results [1]: [max#83] +Functions [1]: [partial_max(csales#80)] +Aggregate Attributes [1]: [max#81] +Results [1]: [max#82] + +(124) RowToColumnar +Input [1]: [max#82] + +(125) CometColumnarExchange +Input [1]: [max#82] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(115) Exchange -Input [1]: [max#83] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] +(126) ColumnarToRow [codegen id : 3] +Input [1]: [max#82] -(116) HashAggregate [codegen id : 3] -Input [1]: [max#83] +(127) HashAggregate [codegen id : 3] +Input [1]: [max#82] Keys: [] -Functions [1]: [max(csales#81)] -Aggregate Attributes [1]: [max(csales#81)#84] -Results [1]: [max(csales#81)#84 AS tpcds_cmax#85] +Functions [1]: [max(csales#80)] +Aggregate Attributes [1]: [max(csales#80)#83] +Results [1]: [max(csales#80)#83 AS tpcds_cmax#84] -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) +Subquery:4 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#70 IN dynamicpruning#71 +BroadcastExchange (132) ++- * ColumnarToRow (131) + +- CometProject (130) + +- CometFilter (129) + +- CometScan parquet spark_catalog.default.date_dim (128) -(117) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#74, d_year#75] +(128) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#73, d_year#74] 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 -(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)) +(129) CometFilter +Input [2]: [d_date_sk#73, d_year#74] +Condition : (d_year#74 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#73)) -(119) CometProject -Input [2]: [d_date_sk#74, d_year#75] -Arguments: [d_date_sk#74], [d_date_sk#74] +(130) CometProject +Input [2]: [d_date_sk#73, d_year#74] +Arguments: [d_date_sk#73], [d_date_sk#73] -(120) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#74] +(131) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#73] -(121) BroadcastExchange -Input [1]: [d_date_sk#74] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] +(132) BroadcastExchange +Input [1]: [d_date_sk#73] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:5 Hosting operator id = 51 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] +Subquery:5 Hosting operator id = 53 Hosting Expression = ReusedSubquery Subquery scalar-subquery#30, [id=#31] -Subquery:6 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 +Subquery:6 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 -Subquery:7 Hosting operator id = 74 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] +Subquery:7 Hosting operator id = 79 Hosting Expression = ReusedSubquery Subquery scalar-subquery#30, [id=#31] 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 d784ed47e..53beb8bb6 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,26 +1,26 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] Union - WholeStageCodegen (14) + WholeStageCodegen (10) 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 (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] - Project [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - SortMergeJoin [cs_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (4) - Sort [cs_bill_customer_sk] - InputAdapter - Exchange [cs_bill_customer_sk] #2 - WholeStageCodegen (3) - Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - BroadcastHashJoin [cs_item_sk,item_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometColumnarExchange [c_last_name,c_first_name] #1 + RowToColumnar + WholeStageCodegen (9) + 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] + Project [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + CometColumnarExchange [cs_bill_customer_sk] #2 + CometProject [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,item_sk] CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -31,153 +31,152 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #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 (1) + CometBroadcastExchange [item_sk] #4 + CometProject [item_sk] + CometFilter [item_sk,cnt] + CometHashAggregate [item_sk,cnt,_groupingexpression,i_item_sk,d_date,count,count(1)] + CometColumnarExchange [_groupingexpression,i_item_sk,d_date] #5 + CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] + CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] + CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] + CometProject [ss_item_sk,d_date] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + CometBroadcastExchange [d_date_sk,d_date] #7 + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + CometBroadcastExchange [i_item_sk,i_item_desc] #8 + CometFilter [i_item_sk,i_item_desc] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + InputAdapter + WholeStageCodegen (3) + Sort [c_customer_sk] + Project [c_customer_sk] + Filter [ssales] + Subquery #3 + WholeStageCodegen (3) + HashAggregate [max] [max(csales),tpcds_cmax,max] + ColumnarToRow + InputAdapter + CometColumnarExchange #11 + RowToColumnar + 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] ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] - CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] - CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [i_item_sk,i_item_desc] #8 - CometFilter [i_item_sk,i_item_desc] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - InputAdapter - WholeStageCodegen (6) - Sort [c_customer_sk] - Project [c_customer_sk] - Filter [ssales] - Subquery #3 - WholeStageCodegen (3) - HashAggregate [max] [max(csales),tpcds_cmax,max] + CometColumnarExchange [c_customer_sk] #12 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometProject [ss_quantity,ss_sales_price,c_customer_sk] + CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] + CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_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 #13 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [c_customer_sk] #10 + CometBroadcastExchange [d_date_sk] #14 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_sk] #9 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometProject [ss_quantity,ss_sales_price,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] + CometProject [ss_customer_sk,ss_quantity,ss_sales_price] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometBroadcastExchange [c_customer_sk] #10 + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk] + InputAdapter + BroadcastExchange #15 + WholeStageCodegen (7) + SortMergeJoin [c_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (4) + ColumnarToRow InputAdapter - 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] #12 - WholeStageCodegen (1) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - ColumnarToRow - InputAdapter - CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] - CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_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 #13 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - 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] #9 - WholeStageCodegen (5) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - ColumnarToRow - InputAdapter - CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] - CometProject [ss_customer_sk,ss_quantity,ss_sales_price] - CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [c_customer_sk] #10 - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk] - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (11) - SortMergeJoin [c_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (8) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #16 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter + CometSort [c_customer_sk,c_first_name,c_last_name] + CometColumnarExchange [c_customer_sk] #16 CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - InputAdapter - 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] #9 - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (28) + InputAdapter + WholeStageCodegen (6) + 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] + ColumnarToRow + InputAdapter + ReusedExchange [c_customer_sk,sum,isEmpty] #9 + InputAdapter + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (20) 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] #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] - Project [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - SortMergeJoin [ws_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (18) - Sort [ws_bill_customer_sk] - InputAdapter - 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 - InputAdapter + ColumnarToRow + InputAdapter + CometColumnarExchange [c_last_name,c_first_name] #17 + RowToColumnar + WholeStageCodegen (19) + 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] + Project [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name] + BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + CometColumnarExchange [ws_bill_customer_sk] #18 + CometProject [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,item_sk] CometFilter [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [item_sk] #4 - InputAdapter - 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] #9 + ReusedExchange [item_sk] #4 + InputAdapter + WholeStageCodegen (13) + 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] + ColumnarToRow + InputAdapter + ReusedExchange [c_customer_sk,sum,isEmpty] #9 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #15 InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #15 - InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/explain.txt index 7241b5ea0..7872cef74 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/explain.txt @@ -1,50 +1,50 @@ == Physical Plan == * Filter (46) +- * HashAggregate (45) - +- Exchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- Exchange (41) + +- * ColumnarToRow (44) + +- CometColumnarExchange (43) + +- RowToColumnar (42) + +- * HashAggregate (41) +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Project (14) - : : : : +- * SortMergeJoin Inner (13) - : : : : :- * Sort (6) - : : : : : +- Exchange (5) - : : : : : +- * ColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- * Sort (12) - : : : : +- Exchange (11) - : : : : +- * ColumnarToRow (10) - : : : : +- CometProject (9) - : : : : +- CometFilter (8) - : : : : +- CometScan parquet spark_catalog.default.store_returns (7) - : : : +- BroadcastExchange (19) - : : : +- * ColumnarToRow (18) - : : : +- CometProject (17) - : : : +- CometFilter (16) - : : : +- CometScan parquet spark_catalog.default.store (15) - : : +- BroadcastExchange (25) - : : +- * ColumnarToRow (24) - : : +- CometFilter (23) - : : +- CometScan parquet spark_catalog.default.item (22) - : +- BroadcastExchange (31) - : +- * ColumnarToRow (30) - : +- CometFilter (29) - : +- CometScan parquet spark_catalog.default.customer (28) - +- BroadcastExchange (37) - +- * ColumnarToRow (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.customer_address (34) + +- * ColumnarToRow (39) + +- CometColumnarExchange (38) + +- RowToColumnar (37) + +- * HashAggregate (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * ColumnarToRow (29) + : +- CometProject (28) + : +- CometBroadcastHashJoin (27) + : :- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometProject (18) + : : : +- CometBroadcastHashJoin (17) + : : : :- CometProject (12) + : : : : +- CometSortMergeJoin (11) + : : : : :- CometSort (5) + : : : : : +- CometColumnarExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometSort (10) + : : : : +- CometColumnarExchange (9) + : : : : +- CometProject (8) + : : : : +- CometFilter (7) + : : : : +- CometScan parquet spark_catalog.default.store_returns (6) + : : : +- CometBroadcastExchange (16) + : : : +- CometProject (15) + : : : +- CometFilter (14) + : : : +- CometScan parquet spark_catalog.default.store (13) + : : +- CometBroadcastExchange (21) + : : +- CometFilter (20) + : : +- CometScan parquet spark_catalog.default.item (19) + : +- CometBroadcastExchange (26) + : +- CometFilter (25) + : +- CometScan parquet spark_catalog.default.customer (24) + +- BroadcastExchange (33) + +- * ColumnarToRow (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.customer_address (30) (1) Scan parquet spark_catalog.default.store_sales @@ -62,277 +62,275 @@ Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND is Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -(4) ColumnarToRow [codegen id : 1] +(4) CometColumnarExchange Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(5) Exchange +(5) CometSort Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] -(6) Sort [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST], false, 0 - -(7) Scan parquet spark_catalog.default.store_returns +(6) Scan parquet spark_catalog.default.store_returns Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(8) CometFilter +(7) CometFilter Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) -(9) CometProject +(8) CometProject Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] -(10) ColumnarToRow [codegen id : 3] -Input [2]: [sr_item_sk#7, sr_ticket_number#8] - -(11) Exchange +(9) CometColumnarExchange Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(12) Sort [codegen id : 4] +(10) CometSort Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST], false, 0 +Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] -(13) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_ticket_number#4, ss_item_sk#1] -Right keys [2]: [sr_ticket_number#8, sr_item_sk#7] -Join type: Inner -Join condition: None +(11) CometSortMergeJoin +Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Right output [2]: [sr_item_sk#7, sr_ticket_number#8] +Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner -(14) Project [codegen id : 9] -Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +(12) CometProject Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -(15) Scan parquet spark_catalog.default.store +(13) Scan parquet spark_catalog.default.store Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(16) CometFilter +(14) CometFilter Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(s_zip#14)) -(17) CometProject +(15) CometProject Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Arguments: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14], [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(18) ColumnarToRow [codegen id : 5] +(16) CometBroadcastExchange Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(19) BroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(20) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#10] -Join type: Inner -Join condition: None +(17) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +Right output [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight -(21) Project [codegen id : 9] -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] +(18) CometProject Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] -(22) Scan parquet spark_catalog.default.item +(19) Scan parquet spark_catalog.default.item Output [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale ), IsNotNull(i_item_sk)] ReadSchema: struct -(23) CometFilter +(20) CometFilter Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Condition : ((isnotnull(i_color#18) AND (i_color#18 = pale )) AND isnotnull(i_item_sk#15)) -(24) ColumnarToRow [codegen id : 6] +(21) CometBroadcastExchange Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Arguments: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(25) BroadcastExchange -Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(22) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] +Right output [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Arguments: [ss_item_sk#1], [i_item_sk#15], Inner, BuildRight -(26) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#15] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 9] -Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +(23) CometProject Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(28) Scan parquet spark_catalog.default.customer +(24) Scan parquet spark_catalog.default.customer Output [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct -(29) CometFilter +(25) CometFilter Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_birth_country#24)) -(30) ColumnarToRow [codegen id : 7] -Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] - -(31) BroadcastExchange +(26) CometBroadcastExchange Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +Arguments: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] -(32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#21] -Join type: Inner -Join condition: None +(27) CometBroadcastHashJoin +Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Right output [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] +Arguments: [ss_customer_sk#2], [c_customer_sk#21], Inner, BuildRight -(33) Project [codegen id : 9] -Output [12]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] +(28) CometProject Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] +Arguments: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24], [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] -(34) Scan parquet spark_catalog.default.customer_address +(29) ColumnarToRow [codegen id : 2] +Input [12]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] + +(30) Scan parquet spark_catalog.default.customer_address Output [3]: [ca_state#25, ca_zip#26, ca_country#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct -(35) CometFilter +(31) CometFilter Input [3]: [ca_state#25, ca_zip#26, ca_country#27] Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) -(36) ColumnarToRow [codegen id : 8] +(32) ColumnarToRow [codegen id : 1] Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -(37) BroadcastExchange +(33) BroadcastExchange Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [plan_id=3] -(38) BroadcastHashJoin [codegen id : 9] +(34) BroadcastHashJoin [codegen id : 2] Left keys [2]: [c_birth_country#24, s_zip#14] Right keys [2]: [upper(ca_country#27), ca_zip#26] Join type: Inner Join condition: None -(39) Project [codegen id : 9] +(35) Project [codegen id : 2] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] Input [15]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24, ca_state#25, ca_zip#26, ca_country#27] -(40) HashAggregate [codegen id : 9] +(36) HashAggregate [codegen id : 2] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum#28] Results [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] -(41) Exchange +(37) RowToColumnar +Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] + +(38) CometColumnarExchange +Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] +Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(39) ColumnarToRow [codegen id : 3] Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] -Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(42) HashAggregate [codegen id : 10] +(40) HashAggregate [codegen id : 3] Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#30] Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#30,17,2) AS netpaid#31] -(43) HashAggregate [codegen id : 10] +(41) HashAggregate [codegen id : 3] Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, netpaid#31] Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11] Functions [1]: [partial_sum(netpaid#31)] Aggregate Attributes [2]: [sum#32, isEmpty#33] Results [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] -(44) Exchange +(42) RowToColumnar Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] -Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(45) HashAggregate [codegen id : 11] +(43) CometColumnarExchange +Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] +Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(44) ColumnarToRow [codegen id : 4] +Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] + +(45) HashAggregate [codegen id : 4] Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11] Functions [1]: [sum(netpaid#31)] Aggregate Attributes [1]: [sum(netpaid#31)#36] Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, sum(netpaid#31)#36 AS paid#37] -(46) Filter [codegen id : 11] +(46) Filter [codegen id : 4] Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, paid#37] Condition : (isnotnull(paid#37) AND (cast(paid#37 as decimal(33,8)) > cast(Subquery scalar-subquery#38, [id=#39] as decimal(33,8)))) ===== Subqueries ===== Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#38, [id=#39] -* HashAggregate (73) -+- Exchange (72) - +- * HashAggregate (71) - +- * HashAggregate (70) - +- Exchange (69) - +- * HashAggregate (68) - +- * Project (67) - +- * BroadcastHashJoin Inner BuildRight (66) - :- * Project (64) - : +- * BroadcastHashJoin Inner BuildRight (63) - : :- * Project (61) - : : +- * BroadcastHashJoin Inner BuildRight (60) - : : :- * Project (55) - : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : :- * Project (52) - : : : : +- * SortMergeJoin Inner (51) - : : : : :- * Sort (48) - : : : : : +- ReusedExchange (47) - : : : : +- * Sort (50) - : : : : +- ReusedExchange (49) - : : : +- ReusedExchange (53) - : : +- BroadcastExchange (59) - : : +- * ColumnarToRow (58) - : : +- CometFilter (57) - : : +- CometScan parquet spark_catalog.default.item (56) - : +- ReusedExchange (62) - +- ReusedExchange (65) - - -(47) ReusedExchange [Reuses operator id: 5] +* HashAggregate (77) ++- * ColumnarToRow (76) + +- CometColumnarExchange (75) + +- RowToColumnar (74) + +- * HashAggregate (73) + +- * HashAggregate (72) + +- * ColumnarToRow (71) + +- CometColumnarExchange (70) + +- RowToColumnar (69) + +- * HashAggregate (68) + +- * Project (67) + +- * BroadcastHashJoin Inner BuildRight (66) + :- * ColumnarToRow (64) + : +- CometProject (63) + : +- CometBroadcastHashJoin (62) + : :- CometProject (60) + : : +- CometBroadcastHashJoin (59) + : : :- CometProject (55) + : : : +- CometBroadcastHashJoin (54) + : : : :- CometProject (52) + : : : : +- CometSortMergeJoin (51) + : : : : :- CometSort (48) + : : : : : +- ReusedExchange (47) + : : : : +- CometSort (50) + : : : : +- ReusedExchange (49) + : : : +- ReusedExchange (53) + : : +- CometBroadcastExchange (58) + : : +- CometFilter (57) + : : +- CometScan parquet spark_catalog.default.item (56) + : +- ReusedExchange (61) + +- ReusedExchange (65) + + +(47) ReusedExchange [Reuses operator id: 4] Output [5]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_ticket_number#43, ss_net_paid#44] -(48) Sort [codegen id : 2] +(48) CometSort Input [5]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_ticket_number#43, ss_net_paid#44] -Arguments: [ss_ticket_number#43 ASC NULLS FIRST, ss_item_sk#40 ASC NULLS FIRST], false, 0 +Arguments: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_ticket_number#43, ss_net_paid#44], [ss_ticket_number#43 ASC NULLS FIRST, ss_item_sk#40 ASC NULLS FIRST] -(49) ReusedExchange [Reuses operator id: 11] +(49) ReusedExchange [Reuses operator id: 9] Output [2]: [sr_item_sk#45, sr_ticket_number#46] -(50) Sort [codegen id : 4] +(50) CometSort Input [2]: [sr_item_sk#45, sr_ticket_number#46] -Arguments: [sr_ticket_number#46 ASC NULLS FIRST, sr_item_sk#45 ASC NULLS FIRST], false, 0 +Arguments: [sr_item_sk#45, sr_ticket_number#46], [sr_ticket_number#46 ASC NULLS FIRST, sr_item_sk#45 ASC NULLS FIRST] -(51) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_ticket_number#43, ss_item_sk#40] -Right keys [2]: [sr_ticket_number#46, sr_item_sk#45] -Join type: Inner -Join condition: None +(51) CometSortMergeJoin +Left output [5]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_ticket_number#43, ss_net_paid#44] +Right output [2]: [sr_item_sk#45, sr_ticket_number#46] +Arguments: [ss_ticket_number#43, ss_item_sk#40], [sr_ticket_number#46, sr_item_sk#45], Inner -(52) Project [codegen id : 9] -Output [4]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_net_paid#44] +(52) CometProject Input [7]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_ticket_number#43, ss_net_paid#44, sr_item_sk#45, sr_ticket_number#46] +Arguments: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_net_paid#44], [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_net_paid#44] -(53) ReusedExchange [Reuses operator id: 19] +(53) ReusedExchange [Reuses operator id: 16] Output [4]: [s_store_sk#47, s_store_name#48, s_state#49, s_zip#50] -(54) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#42] -Right keys [1]: [s_store_sk#47] -Join type: Inner -Join condition: None +(54) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_net_paid#44] +Right output [4]: [s_store_sk#47, s_store_name#48, s_state#49, s_zip#50] +Arguments: [ss_store_sk#42], [s_store_sk#47], Inner, BuildRight -(55) Project [codegen id : 9] -Output [6]: [ss_item_sk#40, ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50] +(55) CometProject Input [8]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_net_paid#44, s_store_sk#47, s_store_name#48, s_state#49, s_zip#50] +Arguments: [ss_item_sk#40, ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50], [ss_item_sk#40, ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50] (56) Scan parquet spark_catalog.default.item Output [6]: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] @@ -345,79 +343,89 @@ ReadSchema: struct -(8) CometFilter +(7) CometFilter Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) -(9) CometProject +(8) CometProject Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] -(10) ColumnarToRow [codegen id : 3] -Input [2]: [sr_item_sk#7, sr_ticket_number#8] - -(11) Exchange +(9) CometColumnarExchange Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(12) Sort [codegen id : 4] +(10) CometSort Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST], false, 0 +Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] -(13) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_ticket_number#4, ss_item_sk#1] -Right keys [2]: [sr_ticket_number#8, sr_item_sk#7] -Join type: Inner -Join condition: None +(11) CometSortMergeJoin +Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Right output [2]: [sr_item_sk#7, sr_ticket_number#8] +Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner -(14) Project [codegen id : 9] -Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +(12) CometProject Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -(15) Scan parquet spark_catalog.default.store +(13) Scan parquet spark_catalog.default.store Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(16) CometFilter +(14) CometFilter Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(s_zip#14)) -(17) CometProject +(15) CometProject Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Arguments: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14], [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(18) ColumnarToRow [codegen id : 5] +(16) CometBroadcastExchange Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(19) BroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(20) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#10] -Join type: Inner -Join condition: None +(17) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +Right output [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight -(21) Project [codegen id : 9] -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] +(18) CometProject Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] -(22) Scan parquet spark_catalog.default.item +(19) Scan parquet spark_catalog.default.item Output [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_color), EqualTo(i_color,chiffon ), IsNotNull(i_item_sk)] ReadSchema: struct -(23) CometFilter +(20) CometFilter Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Condition : ((isnotnull(i_color#18) AND (i_color#18 = chiffon )) AND isnotnull(i_item_sk#15)) -(24) ColumnarToRow [codegen id : 6] +(21) CometBroadcastExchange Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Arguments: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(25) BroadcastExchange -Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(22) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] +Right output [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Arguments: [ss_item_sk#1], [i_item_sk#15], Inner, BuildRight -(26) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#15] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 9] -Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +(23) CometProject Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(28) Scan parquet spark_catalog.default.customer +(24) Scan parquet spark_catalog.default.customer Output [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct -(29) CometFilter +(25) CometFilter Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_birth_country#24)) -(30) ColumnarToRow [codegen id : 7] -Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] - -(31) BroadcastExchange +(26) CometBroadcastExchange Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +Arguments: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] -(32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#21] -Join type: Inner -Join condition: None +(27) CometBroadcastHashJoin +Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Right output [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] +Arguments: [ss_customer_sk#2], [c_customer_sk#21], Inner, BuildRight -(33) Project [codegen id : 9] -Output [12]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] +(28) CometProject Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] +Arguments: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24], [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] -(34) Scan parquet spark_catalog.default.customer_address +(29) ColumnarToRow [codegen id : 2] +Input [12]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] + +(30) Scan parquet spark_catalog.default.customer_address Output [3]: [ca_state#25, ca_zip#26, ca_country#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct -(35) CometFilter +(31) CometFilter Input [3]: [ca_state#25, ca_zip#26, ca_country#27] Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) -(36) ColumnarToRow [codegen id : 8] +(32) ColumnarToRow [codegen id : 1] Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -(37) BroadcastExchange +(33) BroadcastExchange Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [plan_id=3] -(38) BroadcastHashJoin [codegen id : 9] +(34) BroadcastHashJoin [codegen id : 2] Left keys [2]: [c_birth_country#24, s_zip#14] Right keys [2]: [upper(ca_country#27), ca_zip#26] Join type: Inner Join condition: None -(39) Project [codegen id : 9] +(35) Project [codegen id : 2] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] Input [15]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24, ca_state#25, ca_zip#26, ca_country#27] -(40) HashAggregate [codegen id : 9] +(36) HashAggregate [codegen id : 2] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum#28] Results [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] -(41) Exchange +(37) RowToColumnar +Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] + +(38) CometColumnarExchange +Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] +Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(39) ColumnarToRow [codegen id : 3] Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] -Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(42) HashAggregate [codegen id : 10] +(40) HashAggregate [codegen id : 3] Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#30] Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#30,17,2) AS netpaid#31] -(43) HashAggregate [codegen id : 10] +(41) HashAggregate [codegen id : 3] Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, netpaid#31] Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11] Functions [1]: [partial_sum(netpaid#31)] Aggregate Attributes [2]: [sum#32, isEmpty#33] Results [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] -(44) Exchange +(42) RowToColumnar Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] -Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(45) HashAggregate [codegen id : 11] +(43) CometColumnarExchange +Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] +Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(44) ColumnarToRow [codegen id : 4] +Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] + +(45) HashAggregate [codegen id : 4] Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11] Functions [1]: [sum(netpaid#31)] Aggregate Attributes [1]: [sum(netpaid#31)#36] Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, sum(netpaid#31)#36 AS paid#37] -(46) Filter [codegen id : 11] +(46) Filter [codegen id : 4] Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, paid#37] Condition : (isnotnull(paid#37) AND (cast(paid#37 as decimal(33,8)) > cast(Subquery scalar-subquery#38, [id=#39] as decimal(33,8)))) ===== Subqueries ===== Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#38, [id=#39] -* HashAggregate (73) -+- Exchange (72) - +- * HashAggregate (71) - +- * HashAggregate (70) - +- Exchange (69) - +- * HashAggregate (68) - +- * Project (67) - +- * BroadcastHashJoin Inner BuildRight (66) - :- * Project (64) - : +- * BroadcastHashJoin Inner BuildRight (63) - : :- * Project (61) - : : +- * BroadcastHashJoin Inner BuildRight (60) - : : :- * Project (55) - : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : :- * Project (52) - : : : : +- * SortMergeJoin Inner (51) - : : : : :- * Sort (48) - : : : : : +- ReusedExchange (47) - : : : : +- * Sort (50) - : : : : +- ReusedExchange (49) - : : : +- ReusedExchange (53) - : : +- BroadcastExchange (59) - : : +- * ColumnarToRow (58) - : : +- CometFilter (57) - : : +- CometScan parquet spark_catalog.default.item (56) - : +- ReusedExchange (62) - +- ReusedExchange (65) - - -(47) ReusedExchange [Reuses operator id: 5] +* HashAggregate (77) ++- * ColumnarToRow (76) + +- CometColumnarExchange (75) + +- RowToColumnar (74) + +- * HashAggregate (73) + +- * HashAggregate (72) + +- * ColumnarToRow (71) + +- CometColumnarExchange (70) + +- RowToColumnar (69) + +- * HashAggregate (68) + +- * Project (67) + +- * BroadcastHashJoin Inner BuildRight (66) + :- * ColumnarToRow (64) + : +- CometProject (63) + : +- CometBroadcastHashJoin (62) + : :- CometProject (60) + : : +- CometBroadcastHashJoin (59) + : : :- CometProject (55) + : : : +- CometBroadcastHashJoin (54) + : : : :- CometProject (52) + : : : : +- CometSortMergeJoin (51) + : : : : :- CometSort (48) + : : : : : +- ReusedExchange (47) + : : : : +- CometSort (50) + : : : : +- ReusedExchange (49) + : : : +- ReusedExchange (53) + : : +- CometBroadcastExchange (58) + : : +- CometFilter (57) + : : +- CometScan parquet spark_catalog.default.item (56) + : +- ReusedExchange (61) + +- ReusedExchange (65) + + +(47) ReusedExchange [Reuses operator id: 4] Output [5]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_ticket_number#43, ss_net_paid#44] -(48) Sort [codegen id : 2] +(48) CometSort Input [5]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_ticket_number#43, ss_net_paid#44] -Arguments: [ss_ticket_number#43 ASC NULLS FIRST, ss_item_sk#40 ASC NULLS FIRST], false, 0 +Arguments: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_ticket_number#43, ss_net_paid#44], [ss_ticket_number#43 ASC NULLS FIRST, ss_item_sk#40 ASC NULLS FIRST] -(49) ReusedExchange [Reuses operator id: 11] +(49) ReusedExchange [Reuses operator id: 9] Output [2]: [sr_item_sk#45, sr_ticket_number#46] -(50) Sort [codegen id : 4] +(50) CometSort Input [2]: [sr_item_sk#45, sr_ticket_number#46] -Arguments: [sr_ticket_number#46 ASC NULLS FIRST, sr_item_sk#45 ASC NULLS FIRST], false, 0 +Arguments: [sr_item_sk#45, sr_ticket_number#46], [sr_ticket_number#46 ASC NULLS FIRST, sr_item_sk#45 ASC NULLS FIRST] -(51) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_ticket_number#43, ss_item_sk#40] -Right keys [2]: [sr_ticket_number#46, sr_item_sk#45] -Join type: Inner -Join condition: None +(51) CometSortMergeJoin +Left output [5]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_ticket_number#43, ss_net_paid#44] +Right output [2]: [sr_item_sk#45, sr_ticket_number#46] +Arguments: [ss_ticket_number#43, ss_item_sk#40], [sr_ticket_number#46, sr_item_sk#45], Inner -(52) Project [codegen id : 9] -Output [4]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_net_paid#44] +(52) CometProject Input [7]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_ticket_number#43, ss_net_paid#44, sr_item_sk#45, sr_ticket_number#46] +Arguments: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_net_paid#44], [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_net_paid#44] -(53) ReusedExchange [Reuses operator id: 19] +(53) ReusedExchange [Reuses operator id: 16] Output [4]: [s_store_sk#47, s_store_name#48, s_state#49, s_zip#50] -(54) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#42] -Right keys [1]: [s_store_sk#47] -Join type: Inner -Join condition: None +(54) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_net_paid#44] +Right output [4]: [s_store_sk#47, s_store_name#48, s_state#49, s_zip#50] +Arguments: [ss_store_sk#42], [s_store_sk#47], Inner, BuildRight -(55) Project [codegen id : 9] -Output [6]: [ss_item_sk#40, ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50] +(55) CometProject Input [8]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_net_paid#44, s_store_sk#47, s_store_name#48, s_state#49, s_zip#50] +Arguments: [ss_item_sk#40, ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50], [ss_item_sk#40, ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50] (56) Scan parquet spark_catalog.default.item Output [6]: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] @@ -345,79 +343,89 @@ ReadSchema: struct -(44) CometFilter +(46) 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)) -(45) CometProject +(47) CometProject Input [3]: [d_date_sk#19, d_year#20, d_moy#21] Arguments: [d_date_sk#19], [d_date_sk#19] -(46) ColumnarToRow [codegen id : 1] +(48) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] -(47) BroadcastExchange +(49) BroadcastExchange Input [1]: [d_date_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] 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) +BroadcastExchange (54) ++- * ColumnarToRow (53) + +- CometProject (52) + +- CometFilter (51) + +- CometScan parquet spark_catalog.default.date_dim (50) -(48) Scan parquet spark_catalog.default.date_dim +(50) 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 -(49) CometFilter +(51) 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)) -(50) CometProject +(52) CometProject Input [3]: [d_date_sk#22, d_year#23, d_moy#24] Arguments: [d_date_sk#22], [d_date_sk#22] -(51) ColumnarToRow [codegen id : 1] +(53) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(52) BroadcastExchange +(54) BroadcastExchange Input [1]: [d_date_sk#22] 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/q25/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/simplified.txt index 6335e50ff..e435f53e7 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,63 +1,65 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit] 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 (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] - 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,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc] - CometProject [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_sk,s_store_id,s_store_name] - CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,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 [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_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_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,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 [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,cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_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_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] #3 - CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] - 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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] #5 - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_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 [d_date_sk] #6 + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 + RowToColumnar + 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] + 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,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc] + CometProject [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_sk,s_store_id,s_store_name] + CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,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 [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_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_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,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 [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,cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_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_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] + 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_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_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 [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #8 - CometFilter [s_store_sk,s_store_id,s_store_name] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #8 + CometFilter [s_store_sk,s_store_id,s_store_name] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 + CometFilter [i_item_sk,i_item_id,i_item_desc] + 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 d0bc41bcf..b98a5ce41 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 @@ -1,34 +1,36 @@ == Physical Plan == -TakeOrderedAndProject (30) -+- * HashAggregate (29) - +- Exchange (28) - +- * HashAggregate (27) - +- * 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) +TakeOrderedAndProject (32) ++- * HashAggregate (31) + +- * ColumnarToRow (30) + +- CometColumnarExchange (29) + +- RowToColumnar (28) + +- * HashAggregate (27) + +- * 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) (1) Scan parquet spark_catalog.default.catalog_sales @@ -161,50 +163,56 @@ Functions [4]: [partial_avg(cs_quantity#4), partial_avg(UnscaledValue(cs_list_pr 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 +(28) RowToColumnar 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 : 2] +(29) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(30) ColumnarToRow [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] + +(31) 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)#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 +(32) TakeOrderedAndProject 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 ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (35) -+- * ColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan parquet spark_catalog.default.date_dim (31) +BroadcastExchange (37) ++- * ColumnarToRow (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) -(31) Scan parquet spark_catalog.default.date_dim +(33) 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 -(32) CometFilter +(34) 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)) -(33) CometProject +(35) CometProject Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(34) ColumnarToRow [codegen id : 1] +(36) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(35) BroadcastExchange +(37) BroadcastExchange Input [1]: [d_date_sk#14] 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 b98e61704..21f7e6b68 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,42 +1,44 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] 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 (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] - ColumnarToRow - InputAdapter - CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - CometBroadcastHashJoin [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id,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,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_sk,i_item_id] - CometProject [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,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,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk,cd_demo_sk] - CometFilter [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] - 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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #5 - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [p_promo_sk] #6 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_email,p_channel_event] - CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #1 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] + CometBroadcastHashJoin [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id,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,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_sk,i_item_id] + CometProject [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,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,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk,cd_demo_sk] + CometFilter [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] + 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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [cd_demo_sk] #3 + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_item_id] #5 + CometFilter [i_item_sk,i_item_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange [p_promo_sk] #6 + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_email,p_channel_event] + 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 16af0e3a4..9a41c57e1 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 @@ -1,34 +1,36 @@ == Physical Plan == -TakeOrderedAndProject (30) -+- * HashAggregate (29) - +- Exchange (28) - +- * HashAggregate (27) - +- * 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) +TakeOrderedAndProject (32) ++- * HashAggregate (31) + +- * ColumnarToRow (30) + +- CometColumnarExchange (29) + +- RowToColumnar (28) + +- * HashAggregate (27) + +- * 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) (1) Scan parquet spark_catalog.default.store_sales @@ -161,50 +163,56 @@ Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_pr 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 +(28) RowToColumnar 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 : 2] +(29) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(30) ColumnarToRow [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] + +(31) 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)#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 +(32) TakeOrderedAndProject 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 ===== 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 (37) ++- * ColumnarToRow (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) -(31) Scan parquet spark_catalog.default.date_dim +(33) 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 -(32) CometFilter +(34) 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)) -(33) CometProject +(35) CometProject Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(34) ColumnarToRow [codegen id : 1] +(36) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(35) BroadcastExchange +(37) BroadcastExchange Input [1]: [d_date_sk#14] 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 4d4c824bc..894d4215e 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,42 +1,44 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] 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 (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] - 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,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] - CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_state] #5 - CometFilter [s_store_sk,s_state] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id,s_state,spark_grouping_id] #1 + RowToColumnar + 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] + 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,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id] + CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [cd_demo_sk] #3 + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [s_store_sk,s_state] #5 + CometFilter [s_store_sk,s_state] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] + 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/q28/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/explain.txt index de4ab3a2c..615bb79b9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/explain.txt @@ -1,74 +1,98 @@ == Physical Plan == -* BroadcastNestedLoopJoin Inner BuildRight (70) -:- * BroadcastNestedLoopJoin Inner BuildRight (58) -: :- * BroadcastNestedLoopJoin Inner BuildRight (46) -: : :- * BroadcastNestedLoopJoin Inner BuildRight (34) -: : : :- * BroadcastNestedLoopJoin Inner BuildRight (22) -: : : : :- * HashAggregate (10) -: : : : : +- Exchange (9) -: : : : : +- * HashAggregate (8) -: : : : : +- * HashAggregate (7) -: : : : : +- Exchange (6) -: : : : : +- * HashAggregate (5) -: : : : : +- * ColumnarToRow (4) -: : : : : +- CometProject (3) -: : : : : +- CometFilter (2) -: : : : : +- CometScan parquet spark_catalog.default.store_sales (1) -: : : : +- BroadcastExchange (21) -: : : : +- * HashAggregate (20) -: : : : +- Exchange (19) -: : : : +- * HashAggregate (18) -: : : : +- * HashAggregate (17) -: : : : +- Exchange (16) -: : : : +- * HashAggregate (15) -: : : : +- * ColumnarToRow (14) -: : : : +- CometProject (13) -: : : : +- CometFilter (12) -: : : : +- CometScan parquet spark_catalog.default.store_sales (11) -: : : +- BroadcastExchange (33) -: : : +- * HashAggregate (32) -: : : +- Exchange (31) -: : : +- * HashAggregate (30) -: : : +- * HashAggregate (29) -: : : +- Exchange (28) -: : : +- * HashAggregate (27) -: : : +- * ColumnarToRow (26) -: : : +- CometProject (25) -: : : +- CometFilter (24) -: : : +- CometScan parquet spark_catalog.default.store_sales (23) -: : +- BroadcastExchange (45) -: : +- * HashAggregate (44) -: : +- Exchange (43) -: : +- * HashAggregate (42) -: : +- * HashAggregate (41) -: : +- Exchange (40) -: : +- * HashAggregate (39) -: : +- * ColumnarToRow (38) -: : +- CometProject (37) -: : +- CometFilter (36) -: : +- CometScan parquet spark_catalog.default.store_sales (35) -: +- BroadcastExchange (57) -: +- * HashAggregate (56) -: +- Exchange (55) -: +- * HashAggregate (54) -: +- * HashAggregate (53) -: +- Exchange (52) -: +- * HashAggregate (51) -: +- * ColumnarToRow (50) -: +- CometProject (49) -: +- CometFilter (48) -: +- CometScan parquet spark_catalog.default.store_sales (47) -+- BroadcastExchange (69) - +- * HashAggregate (68) - +- Exchange (67) - +- * HashAggregate (66) - +- * HashAggregate (65) - +- Exchange (64) - +- * HashAggregate (63) - +- * ColumnarToRow (62) - +- CometProject (61) - +- CometFilter (60) - +- CometScan parquet spark_catalog.default.store_sales (59) +* BroadcastNestedLoopJoin Inner BuildRight (94) +:- * BroadcastNestedLoopJoin Inner BuildRight (78) +: :- * BroadcastNestedLoopJoin Inner BuildRight (62) +: : :- * BroadcastNestedLoopJoin Inner BuildRight (46) +: : : :- * BroadcastNestedLoopJoin Inner BuildRight (30) +: : : : :- * HashAggregate (14) +: : : : : +- * ColumnarToRow (13) +: : : : : +- CometColumnarExchange (12) +: : : : : +- RowToColumnar (11) +: : : : : +- * HashAggregate (10) +: : : : : +- * HashAggregate (9) +: : : : : +- * ColumnarToRow (8) +: : : : : +- CometColumnarExchange (7) +: : : : : +- RowToColumnar (6) +: : : : : +- * HashAggregate (5) +: : : : : +- * ColumnarToRow (4) +: : : : : +- CometProject (3) +: : : : : +- CometFilter (2) +: : : : : +- CometScan parquet spark_catalog.default.store_sales (1) +: : : : +- BroadcastExchange (29) +: : : : +- * HashAggregate (28) +: : : : +- * ColumnarToRow (27) +: : : : +- CometColumnarExchange (26) +: : : : +- RowToColumnar (25) +: : : : +- * HashAggregate (24) +: : : : +- * HashAggregate (23) +: : : : +- * ColumnarToRow (22) +: : : : +- CometColumnarExchange (21) +: : : : +- RowToColumnar (20) +: : : : +- * HashAggregate (19) +: : : : +- * ColumnarToRow (18) +: : : : +- CometProject (17) +: : : : +- CometFilter (16) +: : : : +- CometScan parquet spark_catalog.default.store_sales (15) +: : : +- BroadcastExchange (45) +: : : +- * HashAggregate (44) +: : : +- * ColumnarToRow (43) +: : : +- CometColumnarExchange (42) +: : : +- RowToColumnar (41) +: : : +- * HashAggregate (40) +: : : +- * HashAggregate (39) +: : : +- * ColumnarToRow (38) +: : : +- CometColumnarExchange (37) +: : : +- RowToColumnar (36) +: : : +- * HashAggregate (35) +: : : +- * ColumnarToRow (34) +: : : +- CometProject (33) +: : : +- CometFilter (32) +: : : +- CometScan parquet spark_catalog.default.store_sales (31) +: : +- BroadcastExchange (61) +: : +- * HashAggregate (60) +: : +- * ColumnarToRow (59) +: : +- CometColumnarExchange (58) +: : +- RowToColumnar (57) +: : +- * HashAggregate (56) +: : +- * HashAggregate (55) +: : +- * ColumnarToRow (54) +: : +- CometColumnarExchange (53) +: : +- RowToColumnar (52) +: : +- * HashAggregate (51) +: : +- * ColumnarToRow (50) +: : +- CometProject (49) +: : +- CometFilter (48) +: : +- CometScan parquet spark_catalog.default.store_sales (47) +: +- BroadcastExchange (77) +: +- * HashAggregate (76) +: +- * ColumnarToRow (75) +: +- CometColumnarExchange (74) +: +- RowToColumnar (73) +: +- * HashAggregate (72) +: +- * HashAggregate (71) +: +- * ColumnarToRow (70) +: +- CometColumnarExchange (69) +: +- RowToColumnar (68) +: +- * HashAggregate (67) +: +- * ColumnarToRow (66) +: +- CometProject (65) +: +- CometFilter (64) +: +- CometScan parquet spark_catalog.default.store_sales (63) ++- BroadcastExchange (93) + +- * HashAggregate (92) + +- * ColumnarToRow (91) + +- CometColumnarExchange (90) + +- RowToColumnar (89) + +- * HashAggregate (88) + +- * HashAggregate (87) + +- * ColumnarToRow (86) + +- CometColumnarExchange (85) + +- RowToColumnar (84) + +- * HashAggregate (83) + +- * ColumnarToRow (82) + +- CometProject (81) + +- CometFilter (80) + +- CometScan parquet spark_catalog.default.store_sales (79) (1) Scan parquet spark_catalog.default.store_sales @@ -96,342 +120,414 @@ Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_li Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#6, count(ss_list_price#3)#7] Results [4]: [ss_list_price#3, sum#8, count#9, count#10] -(6) Exchange +(6) RowToColumnar Input [4]: [ss_list_price#3, sum#8, count#9, count#10] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(7) HashAggregate [codegen id : 2] +(7) CometColumnarExchange +Input [4]: [ss_list_price#3, sum#8, count#9, count#10] +Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(8) ColumnarToRow [codegen id : 2] +Input [4]: [ss_list_price#3, sum#8, count#9, count#10] + +(9) HashAggregate [codegen id : 2] Input [4]: [ss_list_price#3, sum#8, count#9, count#10] Keys [1]: [ss_list_price#3] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#6, count(ss_list_price#3)#7] Results [4]: [ss_list_price#3, sum#8, count#9, count#10] -(8) HashAggregate [codegen id : 2] +(10) HashAggregate [codegen id : 2] Input [4]: [ss_list_price#3, sum#8, count#9, count#10] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#6, count(ss_list_price#3)#7, count(ss_list_price#3)#11] Results [4]: [sum#8, count#9, count#10, count#12] -(9) Exchange +(11) RowToColumnar +Input [4]: [sum#8, count#9, count#10, count#12] + +(12) CometColumnarExchange +Input [4]: [sum#8, count#9, count#10, count#12] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(13) ColumnarToRow [codegen id : 18] Input [4]: [sum#8, count#9, count#10, count#12] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] -(10) HashAggregate [codegen id : 18] +(14) HashAggregate [codegen id : 18] Input [4]: [sum#8, count#9, count#10, count#12] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#6, count(ss_list_price#3)#7, count(ss_list_price#3)#11] Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#6 / 100.0) as decimal(11,6)) AS B1_LP#13, count(ss_list_price#3)#7 AS B1_CNT#14, count(ss_list_price#3)#11 AS B1_CNTD#15] -(11) Scan parquet spark_catalog.default.store_sales +(15) Scan parquet spark_catalog.default.store_sales Output [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,6), LessThanOrEqual(ss_quantity,10), Or(Or(And(GreaterThanOrEqual(ss_list_price,90.00),LessThanOrEqual(ss_list_price,100.00)),And(GreaterThanOrEqual(ss_coupon_amt,2323.00),LessThanOrEqual(ss_coupon_amt,3323.00))),And(GreaterThanOrEqual(ss_wholesale_cost,31.00),LessThanOrEqual(ss_wholesale_cost,51.00)))] ReadSchema: struct -(12) CometFilter +(16) CometFilter Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] Condition : (((isnotnull(ss_quantity#16) AND (ss_quantity#16 >= 6)) AND (ss_quantity#16 <= 10)) AND ((((ss_list_price#18 >= 90.00) AND (ss_list_price#18 <= 100.00)) OR ((ss_coupon_amt#19 >= 2323.00) AND (ss_coupon_amt#19 <= 3323.00))) OR ((ss_wholesale_cost#17 >= 31.00) AND (ss_wholesale_cost#17 <= 51.00)))) -(13) CometProject +(17) CometProject Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] Arguments: [ss_list_price#18], [ss_list_price#18] -(14) ColumnarToRow [codegen id : 3] +(18) ColumnarToRow [codegen id : 3] Input [1]: [ss_list_price#18] -(15) HashAggregate [codegen id : 3] +(19) HashAggregate [codegen id : 3] Input [1]: [ss_list_price#18] Keys [1]: [ss_list_price#18] Functions [2]: [partial_avg(UnscaledValue(ss_list_price#18)), partial_count(ss_list_price#18)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#18))#21, count(ss_list_price#18)#22] Results [4]: [ss_list_price#18, sum#23, count#24, count#25] -(16) Exchange +(20) RowToColumnar Input [4]: [ss_list_price#18, sum#23, count#24, count#25] -Arguments: hashpartitioning(ss_list_price#18, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(17) HashAggregate [codegen id : 4] +(21) CometColumnarExchange +Input [4]: [ss_list_price#18, sum#23, count#24, count#25] +Arguments: hashpartitioning(ss_list_price#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(22) ColumnarToRow [codegen id : 4] +Input [4]: [ss_list_price#18, sum#23, count#24, count#25] + +(23) HashAggregate [codegen id : 4] Input [4]: [ss_list_price#18, sum#23, count#24, count#25] Keys [1]: [ss_list_price#18] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#18))#21, count(ss_list_price#18)#22] Results [4]: [ss_list_price#18, sum#23, count#24, count#25] -(18) HashAggregate [codegen id : 4] +(24) HashAggregate [codegen id : 4] Input [4]: [ss_list_price#18, sum#23, count#24, count#25] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18), partial_count(distinct ss_list_price#18)] Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#18))#21, count(ss_list_price#18)#22, count(ss_list_price#18)#26] Results [4]: [sum#23, count#24, count#25, count#27] -(19) Exchange +(25) RowToColumnar Input [4]: [sum#23, count#24, count#25, count#27] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] -(20) HashAggregate [codegen id : 5] +(26) CometColumnarExchange +Input [4]: [sum#23, count#24, count#25, count#27] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(27) ColumnarToRow [codegen id : 5] +Input [4]: [sum#23, count#24, count#25, count#27] + +(28) HashAggregate [codegen id : 5] Input [4]: [sum#23, count#24, count#25, count#27] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#18)), count(ss_list_price#18), count(distinct ss_list_price#18)] Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#18))#21, count(ss_list_price#18)#22, count(ss_list_price#18)#26] Results [3]: [cast((avg(UnscaledValue(ss_list_price#18))#21 / 100.0) as decimal(11,6)) AS B2_LP#28, count(ss_list_price#18)#22 AS B2_CNT#29, count(ss_list_price#18)#26 AS B2_CNTD#30] -(21) BroadcastExchange +(29) BroadcastExchange Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] Arguments: IdentityBroadcastMode, [plan_id=5] -(22) BroadcastNestedLoopJoin [codegen id : 18] +(30) BroadcastNestedLoopJoin [codegen id : 18] Join type: Inner Join condition: None -(23) Scan parquet spark_catalog.default.store_sales +(31) Scan parquet spark_catalog.default.store_sales Output [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,11), LessThanOrEqual(ss_quantity,15), Or(Or(And(GreaterThanOrEqual(ss_list_price,142.00),LessThanOrEqual(ss_list_price,152.00)),And(GreaterThanOrEqual(ss_coupon_amt,12214.00),LessThanOrEqual(ss_coupon_amt,13214.00))),And(GreaterThanOrEqual(ss_wholesale_cost,79.00),LessThanOrEqual(ss_wholesale_cost,99.00)))] ReadSchema: struct -(24) CometFilter +(32) CometFilter Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] Condition : (((isnotnull(ss_quantity#31) AND (ss_quantity#31 >= 11)) AND (ss_quantity#31 <= 15)) AND ((((ss_list_price#33 >= 142.00) AND (ss_list_price#33 <= 152.00)) OR ((ss_coupon_amt#34 >= 12214.00) AND (ss_coupon_amt#34 <= 13214.00))) OR ((ss_wholesale_cost#32 >= 79.00) AND (ss_wholesale_cost#32 <= 99.00)))) -(25) CometProject +(33) CometProject Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] Arguments: [ss_list_price#33], [ss_list_price#33] -(26) ColumnarToRow [codegen id : 6] +(34) ColumnarToRow [codegen id : 6] Input [1]: [ss_list_price#33] -(27) HashAggregate [codegen id : 6] +(35) HashAggregate [codegen id : 6] Input [1]: [ss_list_price#33] Keys [1]: [ss_list_price#33] Functions [2]: [partial_avg(UnscaledValue(ss_list_price#33)), partial_count(ss_list_price#33)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#33))#36, count(ss_list_price#33)#37] Results [4]: [ss_list_price#33, sum#38, count#39, count#40] -(28) Exchange +(36) RowToColumnar +Input [4]: [ss_list_price#33, sum#38, count#39, count#40] + +(37) CometColumnarExchange Input [4]: [ss_list_price#33, sum#38, count#39, count#40] -Arguments: hashpartitioning(ss_list_price#33, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Arguments: hashpartitioning(ss_list_price#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(29) HashAggregate [codegen id : 7] +(38) ColumnarToRow [codegen id : 7] +Input [4]: [ss_list_price#33, sum#38, count#39, count#40] + +(39) HashAggregate [codegen id : 7] Input [4]: [ss_list_price#33, sum#38, count#39, count#40] Keys [1]: [ss_list_price#33] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#33))#36, count(ss_list_price#33)#37] Results [4]: [ss_list_price#33, sum#38, count#39, count#40] -(30) HashAggregate [codegen id : 7] +(40) HashAggregate [codegen id : 7] Input [4]: [ss_list_price#33, sum#38, count#39, count#40] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33), partial_count(distinct ss_list_price#33)] Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#33))#36, count(ss_list_price#33)#37, count(ss_list_price#33)#41] Results [4]: [sum#38, count#39, count#40, count#42] -(31) Exchange +(41) RowToColumnar +Input [4]: [sum#38, count#39, count#40, count#42] + +(42) CometColumnarExchange Input [4]: [sum#38, count#39, count#40, count#42] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(32) HashAggregate [codegen id : 8] +(43) ColumnarToRow [codegen id : 8] +Input [4]: [sum#38, count#39, count#40, count#42] + +(44) HashAggregate [codegen id : 8] Input [4]: [sum#38, count#39, count#40, count#42] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#33)), count(ss_list_price#33), count(distinct ss_list_price#33)] Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#33))#36, count(ss_list_price#33)#37, count(ss_list_price#33)#41] Results [3]: [cast((avg(UnscaledValue(ss_list_price#33))#36 / 100.0) as decimal(11,6)) AS B3_LP#43, count(ss_list_price#33)#37 AS B3_CNT#44, count(ss_list_price#33)#41 AS B3_CNTD#45] -(33) BroadcastExchange +(45) BroadcastExchange Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] Arguments: IdentityBroadcastMode, [plan_id=8] -(34) BroadcastNestedLoopJoin [codegen id : 18] +(46) BroadcastNestedLoopJoin [codegen id : 18] Join type: Inner Join condition: None -(35) Scan parquet spark_catalog.default.store_sales +(47) Scan parquet spark_catalog.default.store_sales Output [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,16), LessThanOrEqual(ss_quantity,20), Or(Or(And(GreaterThanOrEqual(ss_list_price,135.00),LessThanOrEqual(ss_list_price,145.00)),And(GreaterThanOrEqual(ss_coupon_amt,6071.00),LessThanOrEqual(ss_coupon_amt,7071.00))),And(GreaterThanOrEqual(ss_wholesale_cost,38.00),LessThanOrEqual(ss_wholesale_cost,58.00)))] ReadSchema: struct -(36) CometFilter +(48) CometFilter Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] Condition : (((isnotnull(ss_quantity#46) AND (ss_quantity#46 >= 16)) AND (ss_quantity#46 <= 20)) AND ((((ss_list_price#48 >= 135.00) AND (ss_list_price#48 <= 145.00)) OR ((ss_coupon_amt#49 >= 6071.00) AND (ss_coupon_amt#49 <= 7071.00))) OR ((ss_wholesale_cost#47 >= 38.00) AND (ss_wholesale_cost#47 <= 58.00)))) -(37) CometProject +(49) CometProject Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] Arguments: [ss_list_price#48], [ss_list_price#48] -(38) ColumnarToRow [codegen id : 9] +(50) ColumnarToRow [codegen id : 9] Input [1]: [ss_list_price#48] -(39) HashAggregate [codegen id : 9] +(51) HashAggregate [codegen id : 9] Input [1]: [ss_list_price#48] Keys [1]: [ss_list_price#48] Functions [2]: [partial_avg(UnscaledValue(ss_list_price#48)), partial_count(ss_list_price#48)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#48))#51, count(ss_list_price#48)#52] Results [4]: [ss_list_price#48, sum#53, count#54, count#55] -(40) Exchange +(52) RowToColumnar Input [4]: [ss_list_price#48, sum#53, count#54, count#55] -Arguments: hashpartitioning(ss_list_price#48, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(41) HashAggregate [codegen id : 10] +(53) CometColumnarExchange +Input [4]: [ss_list_price#48, sum#53, count#54, count#55] +Arguments: hashpartitioning(ss_list_price#48, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(54) ColumnarToRow [codegen id : 10] +Input [4]: [ss_list_price#48, sum#53, count#54, count#55] + +(55) HashAggregate [codegen id : 10] Input [4]: [ss_list_price#48, sum#53, count#54, count#55] Keys [1]: [ss_list_price#48] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#48))#51, count(ss_list_price#48)#52] Results [4]: [ss_list_price#48, sum#53, count#54, count#55] -(42) HashAggregate [codegen id : 10] +(56) HashAggregate [codegen id : 10] Input [4]: [ss_list_price#48, sum#53, count#54, count#55] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48), partial_count(distinct ss_list_price#48)] Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#48))#51, count(ss_list_price#48)#52, count(ss_list_price#48)#56] Results [4]: [sum#53, count#54, count#55, count#57] -(43) Exchange +(57) RowToColumnar +Input [4]: [sum#53, count#54, count#55, count#57] + +(58) CometColumnarExchange +Input [4]: [sum#53, count#54, count#55, count#57] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(59) ColumnarToRow [codegen id : 11] Input [4]: [sum#53, count#54, count#55, count#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] -(44) HashAggregate [codegen id : 11] +(60) HashAggregate [codegen id : 11] Input [4]: [sum#53, count#54, count#55, count#57] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#48)), count(ss_list_price#48), count(distinct ss_list_price#48)] Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#48))#51, count(ss_list_price#48)#52, count(ss_list_price#48)#56] Results [3]: [cast((avg(UnscaledValue(ss_list_price#48))#51 / 100.0) as decimal(11,6)) AS B4_LP#58, count(ss_list_price#48)#52 AS B4_CNT#59, count(ss_list_price#48)#56 AS B4_CNTD#60] -(45) BroadcastExchange +(61) BroadcastExchange Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] Arguments: IdentityBroadcastMode, [plan_id=11] -(46) BroadcastNestedLoopJoin [codegen id : 18] +(62) BroadcastNestedLoopJoin [codegen id : 18] Join type: Inner Join condition: None -(47) Scan parquet spark_catalog.default.store_sales +(63) Scan parquet spark_catalog.default.store_sales Output [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,25), Or(Or(And(GreaterThanOrEqual(ss_list_price,122.00),LessThanOrEqual(ss_list_price,132.00)),And(GreaterThanOrEqual(ss_coupon_amt,836.00),LessThanOrEqual(ss_coupon_amt,1836.00))),And(GreaterThanOrEqual(ss_wholesale_cost,17.00),LessThanOrEqual(ss_wholesale_cost,37.00)))] ReadSchema: struct -(48) CometFilter +(64) CometFilter Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] Condition : (((isnotnull(ss_quantity#61) AND (ss_quantity#61 >= 21)) AND (ss_quantity#61 <= 25)) AND ((((ss_list_price#63 >= 122.00) AND (ss_list_price#63 <= 132.00)) OR ((ss_coupon_amt#64 >= 836.00) AND (ss_coupon_amt#64 <= 1836.00))) OR ((ss_wholesale_cost#62 >= 17.00) AND (ss_wholesale_cost#62 <= 37.00)))) -(49) CometProject +(65) CometProject Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] Arguments: [ss_list_price#63], [ss_list_price#63] -(50) ColumnarToRow [codegen id : 12] +(66) ColumnarToRow [codegen id : 12] Input [1]: [ss_list_price#63] -(51) HashAggregate [codegen id : 12] +(67) HashAggregate [codegen id : 12] Input [1]: [ss_list_price#63] Keys [1]: [ss_list_price#63] Functions [2]: [partial_avg(UnscaledValue(ss_list_price#63)), partial_count(ss_list_price#63)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#63))#66, count(ss_list_price#63)#67] Results [4]: [ss_list_price#63, sum#68, count#69, count#70] -(52) Exchange +(68) RowToColumnar +Input [4]: [ss_list_price#63, sum#68, count#69, count#70] + +(69) CometColumnarExchange Input [4]: [ss_list_price#63, sum#68, count#69, count#70] -Arguments: hashpartitioning(ss_list_price#63, 5), ENSURE_REQUIREMENTS, [plan_id=12] +Arguments: hashpartitioning(ss_list_price#63, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(53) HashAggregate [codegen id : 13] +(70) ColumnarToRow [codegen id : 13] +Input [4]: [ss_list_price#63, sum#68, count#69, count#70] + +(71) HashAggregate [codegen id : 13] Input [4]: [ss_list_price#63, sum#68, count#69, count#70] Keys [1]: [ss_list_price#63] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#63))#66, count(ss_list_price#63)#67] Results [4]: [ss_list_price#63, sum#68, count#69, count#70] -(54) HashAggregate [codegen id : 13] +(72) HashAggregate [codegen id : 13] Input [4]: [ss_list_price#63, sum#68, count#69, count#70] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63), partial_count(distinct ss_list_price#63)] Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#63))#66, count(ss_list_price#63)#67, count(ss_list_price#63)#71] Results [4]: [sum#68, count#69, count#70, count#72] -(55) Exchange +(73) RowToColumnar +Input [4]: [sum#68, count#69, count#70, count#72] + +(74) CometColumnarExchange Input [4]: [sum#68, count#69, count#70, count#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(56) HashAggregate [codegen id : 14] +(75) ColumnarToRow [codegen id : 14] +Input [4]: [sum#68, count#69, count#70, count#72] + +(76) HashAggregate [codegen id : 14] Input [4]: [sum#68, count#69, count#70, count#72] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#63)), count(ss_list_price#63), count(distinct ss_list_price#63)] Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#63))#66, count(ss_list_price#63)#67, count(ss_list_price#63)#71] Results [3]: [cast((avg(UnscaledValue(ss_list_price#63))#66 / 100.0) as decimal(11,6)) AS B5_LP#73, count(ss_list_price#63)#67 AS B5_CNT#74, count(ss_list_price#63)#71 AS B5_CNTD#75] -(57) BroadcastExchange +(77) BroadcastExchange Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] Arguments: IdentityBroadcastMode, [plan_id=14] -(58) BroadcastNestedLoopJoin [codegen id : 18] +(78) BroadcastNestedLoopJoin [codegen id : 18] Join type: Inner Join condition: None -(59) Scan parquet spark_catalog.default.store_sales +(79) Scan parquet spark_catalog.default.store_sales Output [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,26), LessThanOrEqual(ss_quantity,30), Or(Or(And(GreaterThanOrEqual(ss_list_price,154.00),LessThanOrEqual(ss_list_price,164.00)),And(GreaterThanOrEqual(ss_coupon_amt,7326.00),LessThanOrEqual(ss_coupon_amt,8326.00))),And(GreaterThanOrEqual(ss_wholesale_cost,7.00),LessThanOrEqual(ss_wholesale_cost,27.00)))] ReadSchema: struct -(60) CometFilter +(80) CometFilter Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] Condition : (((isnotnull(ss_quantity#76) AND (ss_quantity#76 >= 26)) AND (ss_quantity#76 <= 30)) AND ((((ss_list_price#78 >= 154.00) AND (ss_list_price#78 <= 164.00)) OR ((ss_coupon_amt#79 >= 7326.00) AND (ss_coupon_amt#79 <= 8326.00))) OR ((ss_wholesale_cost#77 >= 7.00) AND (ss_wholesale_cost#77 <= 27.00)))) -(61) CometProject +(81) CometProject Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] Arguments: [ss_list_price#78], [ss_list_price#78] -(62) ColumnarToRow [codegen id : 15] +(82) ColumnarToRow [codegen id : 15] Input [1]: [ss_list_price#78] -(63) HashAggregate [codegen id : 15] +(83) HashAggregate [codegen id : 15] Input [1]: [ss_list_price#78] Keys [1]: [ss_list_price#78] Functions [2]: [partial_avg(UnscaledValue(ss_list_price#78)), partial_count(ss_list_price#78)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#78))#81, count(ss_list_price#78)#82] Results [4]: [ss_list_price#78, sum#83, count#84, count#85] -(64) Exchange +(84) RowToColumnar +Input [4]: [ss_list_price#78, sum#83, count#84, count#85] + +(85) CometColumnarExchange +Input [4]: [ss_list_price#78, sum#83, count#84, count#85] +Arguments: hashpartitioning(ss_list_price#78, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] + +(86) ColumnarToRow [codegen id : 16] Input [4]: [ss_list_price#78, sum#83, count#84, count#85] -Arguments: hashpartitioning(ss_list_price#78, 5), ENSURE_REQUIREMENTS, [plan_id=15] -(65) HashAggregate [codegen id : 16] +(87) HashAggregate [codegen id : 16] Input [4]: [ss_list_price#78, sum#83, count#84, count#85] Keys [1]: [ss_list_price#78] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#78))#81, count(ss_list_price#78)#82] Results [4]: [ss_list_price#78, sum#83, count#84, count#85] -(66) HashAggregate [codegen id : 16] +(88) HashAggregate [codegen id : 16] Input [4]: [ss_list_price#78, sum#83, count#84, count#85] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78), partial_count(distinct ss_list_price#78)] Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#78))#81, count(ss_list_price#78)#82, count(ss_list_price#78)#86] Results [4]: [sum#83, count#84, count#85, count#87] -(67) Exchange +(89) RowToColumnar +Input [4]: [sum#83, count#84, count#85, count#87] + +(90) CometColumnarExchange +Input [4]: [sum#83, count#84, count#85, count#87] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] + +(91) ColumnarToRow [codegen id : 17] Input [4]: [sum#83, count#84, count#85, count#87] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=16] -(68) HashAggregate [codegen id : 17] +(92) HashAggregate [codegen id : 17] Input [4]: [sum#83, count#84, count#85, count#87] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#78)), count(ss_list_price#78), count(distinct ss_list_price#78)] Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#78))#81, count(ss_list_price#78)#82, count(ss_list_price#78)#86] Results [3]: [cast((avg(UnscaledValue(ss_list_price#78))#81 / 100.0) as decimal(11,6)) AS B6_LP#88, count(ss_list_price#78)#82 AS B6_CNT#89, count(ss_list_price#78)#86 AS B6_CNTD#90] -(69) BroadcastExchange +(93) BroadcastExchange Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] Arguments: IdentityBroadcastMode, [plan_id=17] -(70) BroadcastNestedLoopJoin [codegen id : 18] +(94) BroadcastNestedLoopJoin [codegen id : 18] Join type: Inner Join condition: None diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/simplified.txt index 2136886fe..7c4823efd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/simplified.txt @@ -5,107 +5,131 @@ WholeStageCodegen (18) BroadcastNestedLoopJoin BroadcastNestedLoopJoin HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B1_LP,B1_CNT,B1_CNTD,sum,count,count,count] - InputAdapter - Exchange #1 - WholeStageCodegen (2) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #2 - WholeStageCodegen (1) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - ColumnarToRow - InputAdapter - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + ColumnarToRow + InputAdapter + CometColumnarExchange #1 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + ColumnarToRow + InputAdapter + CometColumnarExchange [ss_list_price] #2 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + ColumnarToRow + InputAdapter + CometProject [ss_list_price] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (5) HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B2_LP,B2_CNT,B2_CNTD,sum,count,count,count] - InputAdapter - Exchange #4 - WholeStageCodegen (4) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #5 - WholeStageCodegen (3) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - ColumnarToRow - InputAdapter - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + ColumnarToRow + InputAdapter + CometColumnarExchange #4 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + ColumnarToRow + InputAdapter + CometColumnarExchange [ss_list_price] #5 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + ColumnarToRow + InputAdapter + CometProject [ss_list_price] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #6 WholeStageCodegen (8) HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B3_LP,B3_CNT,B3_CNTD,sum,count,count,count] - InputAdapter - Exchange #7 - WholeStageCodegen (7) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #8 - WholeStageCodegen (6) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - ColumnarToRow - InputAdapter - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + ColumnarToRow + InputAdapter + CometColumnarExchange #7 + RowToColumnar + WholeStageCodegen (7) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + ColumnarToRow + InputAdapter + CometColumnarExchange [ss_list_price] #8 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + ColumnarToRow + InputAdapter + CometProject [ss_list_price] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #9 WholeStageCodegen (11) HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B4_LP,B4_CNT,B4_CNTD,sum,count,count,count] - InputAdapter - Exchange #10 - WholeStageCodegen (10) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #11 - WholeStageCodegen (9) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - ColumnarToRow - InputAdapter - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + ColumnarToRow + InputAdapter + CometColumnarExchange #10 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + ColumnarToRow + InputAdapter + CometColumnarExchange [ss_list_price] #11 + RowToColumnar + WholeStageCodegen (9) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + ColumnarToRow + InputAdapter + CometProject [ss_list_price] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #12 WholeStageCodegen (14) HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B5_LP,B5_CNT,B5_CNTD,sum,count,count,count] - InputAdapter - Exchange #13 - WholeStageCodegen (13) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #14 - WholeStageCodegen (12) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - ColumnarToRow - InputAdapter - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + ColumnarToRow + InputAdapter + CometColumnarExchange #13 + RowToColumnar + WholeStageCodegen (13) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + ColumnarToRow + InputAdapter + CometColumnarExchange [ss_list_price] #14 + RowToColumnar + WholeStageCodegen (12) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + ColumnarToRow + InputAdapter + CometProject [ss_list_price] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #15 WholeStageCodegen (17) HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B6_LP,B6_CNT,B6_CNTD,sum,count,count,count] - InputAdapter - Exchange #16 - WholeStageCodegen (16) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #17 - WholeStageCodegen (15) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - ColumnarToRow - InputAdapter - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + ColumnarToRow + InputAdapter + CometColumnarExchange #16 + RowToColumnar + WholeStageCodegen (16) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + ColumnarToRow + InputAdapter + CometColumnarExchange [ss_list_price] #17 + RowToColumnar + WholeStageCodegen (15) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + ColumnarToRow + InputAdapter + CometProject [ss_list_price] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] 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 1bf9ac34d..f6ab979fb 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,49 +1,51 @@ == Physical Plan == -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) +TakeOrderedAndProject (47) ++- * HashAggregate (46) + +- * ColumnarToRow (45) + +- CometColumnarExchange (44) + +- RowToColumnar (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 @@ -250,110 +252,116 @@ Functions [3]: [partial_sum(ss_quantity#5), partial_sum(sr_return_quantity#11), 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] -(43) Exchange +(43) RowToColumnar 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] -(44) HashAggregate [codegen id : 2] +(44) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(45) ColumnarToRow [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] + +(46) 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)#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] -(45) TakeOrderedAndProject +(47) 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 (50) -+- * ColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan parquet spark_catalog.default.date_dim (46) +BroadcastExchange (52) ++- * ColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) -(46) Scan parquet spark_catalog.default.date_dim +(48) 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 -(47) CometFilter +(49) 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)) -(48) CometProject +(50) CometProject 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] +(51) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] -(50) BroadcastExchange +(52) BroadcastExchange Input [1]: [d_date_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (55) -+- * ColumnarToRow (54) - +- CometProject (53) - +- CometFilter (52) - +- CometScan parquet spark_catalog.default.date_dim (51) +BroadcastExchange (57) ++- * ColumnarToRow (56) + +- CometProject (55) + +- CometFilter (54) + +- CometScan parquet spark_catalog.default.date_dim (53) -(51) Scan parquet spark_catalog.default.date_dim +(53) 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 -(52) CometFilter +(54) 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)) -(53) CometProject +(55) CometProject 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] +(56) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(55) BroadcastExchange +(57) 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 = 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) +BroadcastExchange (62) ++- * ColumnarToRow (61) + +- CometProject (60) + +- CometFilter (59) + +- CometScan parquet spark_catalog.default.date_dim (58) -(56) Scan parquet spark_catalog.default.date_dim +(58) 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 +(59) 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 +(60) CometProject Input [2]: [d_date_sk#25, d_year#26] Arguments: [d_date_sk#25], [d_date_sk#25] -(59) ColumnarToRow [codegen id : 1] +(61) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] -(60) BroadcastExchange +(62) 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 ee1f435d5..e2c8ade5f 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,73 +1,75 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity] 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 (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] - 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,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc] - CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_store_id,s_store_name] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,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 [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_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_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_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 [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,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 - CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - 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 + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 + RowToColumnar + 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] + 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,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc] + CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_store_id,s_store_name] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,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 [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_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_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_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 [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,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + 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_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #7 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #7 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk] #8 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #8 + CometBroadcastExchange [d_date_sk] #9 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #9 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #10 - CometFilter [s_store_sk,s_store_id,s_store_name] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #11 - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #10 + CometFilter [s_store_sk,s_store_id,s_store_name] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #11 + CometFilter [i_item_sk,i_item_id,i_item_desc] + 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 75546d3e5..8e804be1f 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,23 +1,25 @@ == Physical Plan == -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) +TakeOrderedAndProject (21) ++- * HashAggregate (20) + +- * ColumnarToRow (19) + +- CometColumnarExchange (18) + +- RowToColumnar (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 @@ -98,18 +100,24 @@ 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] -(17) Exchange +(17) RowToColumnar 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=1] -(18) HashAggregate [codegen id : 2] +(18) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(19) ColumnarToRow [codegen id : 2] +Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] + +(20) 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] -(19) TakeOrderedAndProject +(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] 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 36bb17a16..ffb864d3f 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,23 +1,25 @@ TakeOrderedAndProject [d_year,sum_agg,brand_id,brand] 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 (1) - HashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] - CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id] + ColumnarToRow + InputAdapter + CometColumnarExchange [d_year,i_brand,i_brand_id] #1 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] + CometProject [d_year,ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometProject [d_date_sk,d_year] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 + CometProject [i_item_sk,i_brand_id,i_brand] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id] + 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 004e04634..83890381c 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,55 +1,61 @@ == Physical Plan == -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) +TakeOrderedAndProject (57) ++- * Project (56) + +- * BroadcastHashJoin Inner BuildRight (55) + :- * Project (49) + : +- * BroadcastHashJoin Inner BuildRight (48) + : :- * Project (43) + : : +- * BroadcastHashJoin Inner BuildRight (42) + : : :- * Filter (20) + : : : +- * HashAggregate (19) + : : : +- * ColumnarToRow (18) + : : : +- CometColumnarExchange (17) + : : : +- RowToColumnar (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 (41) + : : +- * Filter (40) + : : +- * HashAggregate (39) + : : +- * ColumnarToRow (38) + : : +- CometColumnarExchange (37) + : : +- RowToColumnar (36) + : : +- * HashAggregate (35) + : : +- * HashAggregate (34) + : : +- * ColumnarToRow (33) + : : +- CometColumnarExchange (32) + : : +- RowToColumnar (31) + : : +- * HashAggregate (30) + : : +- * ColumnarToRow (29) + : : +- CometProject (28) + : : +- CometBroadcastHashJoin (27) + : : :- CometProject (25) + : : : +- CometBroadcastHashJoin (24) + : : : :- CometFilter (22) + : : : : +- CometScan parquet spark_catalog.default.web_returns (21) + : : : +- ReusedExchange (23) + : : +- ReusedExchange (26) + : +- BroadcastExchange (47) + : +- * ColumnarToRow (46) + : +- CometFilter (45) + : +- CometScan parquet spark_catalog.default.customer (44) + +- BroadcastExchange (54) + +- * ColumnarToRow (53) + +- CometProject (52) + +- CometFilter (51) + +- CometScan parquet spark_catalog.default.customer_address (50) (1) Scan parquet spark_catalog.default.web_returns @@ -126,22 +132,28 @@ Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] Aggregate Attributes [1]: [sum#10] Results [3]: [wr_returning_customer_sk#1, ca_state#9, sum#11] -(16) Exchange +(16) RowToColumnar 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] -(17) HashAggregate [codegen id : 7] +(17) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(18) ColumnarToRow [codegen id : 7] +Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#11] + +(19) 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))#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] -(18) Filter [codegen id : 7] +(20) Filter [codegen id : 7] Input [3]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15] Condition : isnotnull(ctr_total_return#15) -(19) Scan parquet spark_catalog.default.web_returns +(21) 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 [] @@ -149,187 +161,199 @@ PartitionFilters: [isnotnull(wr_returned_date_sk#19), dynamicpruningexpression(w PushedFilters: [IsNotNull(wr_returning_addr_sk)] ReadSchema: struct -(20) CometFilter +(22) 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] +(23) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#21] -(22) CometBroadcastHashJoin +(24) 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 +(25) 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] +(26) ReusedExchange [Reuses operator id: 11] Output [2]: [ca_address_sk#22, ca_state#23] -(25) CometBroadcastHashJoin +(27) 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 +(28) 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] +(29) ColumnarToRow [codegen id : 2] Input [3]: [wr_returning_customer_sk#16, wr_return_amt#18, ca_state#23] -(28) HashAggregate [codegen id : 2] +(30) 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 +(31) RowToColumnar +Input [3]: [wr_returning_customer_sk#16, ca_state#23, sum#25] + +(32) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] + +(33) ColumnarToRow [codegen id : 3] 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] +(34) 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] +(35) 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 +(36) RowToColumnar +Input [3]: [ctr_state#26, sum#30, count#31] + +(37) CometColumnarExchange Input [3]: [ctr_state#26, sum#30, count#31] -Arguments: hashpartitioning(ctr_state#26, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(ctr_state#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(33) HashAggregate [codegen id : 4] +(38) ColumnarToRow [codegen id : 4] +Input [3]: [ctr_state#26, sum#30, count#31] + +(39) 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] +(40) 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 +(41) 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] +(42) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ctr_state#14] Right keys [1]: [ctr_state#26] Join type: Inner Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#33) -(37) Project [codegen id : 7] +(43) 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] -(38) Scan parquet spark_catalog.default.customer +(44) 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 -(39) CometFilter +(45) 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)) -(40) ColumnarToRow [codegen id : 5] +(46) 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] -(41) BroadcastExchange +(47) 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] -(42) BroadcastHashJoin [codegen id : 7] +(48) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ctr_customer_sk#13] Right keys [1]: [c_customer_sk#34] Join type: Inner Join condition: None -(43) Project [codegen id : 7] +(49) 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] -(44) Scan parquet spark_catalog.default.customer_address +(50) 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 -(45) CometFilter +(51) 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)) -(46) CometProject +(52) CometProject Input [2]: [ca_address_sk#48, ca_state#49] Arguments: [ca_address_sk#48], [ca_address_sk#48] -(47) ColumnarToRow [codegen id : 6] +(53) ColumnarToRow [codegen id : 6] Input [1]: [ca_address_sk#48] -(48) BroadcastExchange +(54) BroadcastExchange Input [1]: [ca_address_sk#48] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(49) BroadcastHashJoin [codegen id : 7] +(55) 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 -(50) Project [codegen id : 7] +(56) 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] -(51) TakeOrderedAndProject +(57) 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 (56) -+- * ColumnarToRow (55) - +- CometProject (54) - +- CometFilter (53) - +- CometScan parquet spark_catalog.default.date_dim (52) +BroadcastExchange (62) ++- * ColumnarToRow (61) + +- CometProject (60) + +- CometFilter (59) + +- CometScan parquet spark_catalog.default.date_dim (58) -(52) Scan parquet spark_catalog.default.date_dim +(58) 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 -(53) CometFilter +(59) 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)) -(54) CometProject +(60) CometProject Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(55) ColumnarToRow [codegen id : 1] +(61) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(56) BroadcastExchange +(62) 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 = 19 Hosting Expression = wr_returned_date_sk#19 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 21 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 c4e59b49a..0af6e3103 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 @@ -8,58 +8,64 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] Filter [ctr_total_return] 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 (1) - HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] - CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] - CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [ca_address_sk,ca_state] #4 - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + ColumnarToRow + InputAdapter + CometColumnarExchange [wr_returning_customer_sk,ca_state] #1 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] + CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] + CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [ca_address_sk,ca_state] #4 + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter 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] #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] #7 - WholeStageCodegen (2) - HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] - CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] - CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_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 - ReusedExchange [d_date_sk] #3 - ReusedExchange [ca_address_sk,ca_state] #4 + ColumnarToRow + InputAdapter + CometColumnarExchange [ctr_state] #6 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [wr_returning_customer_sk,ca_state] #7 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] + CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] + CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_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 + ReusedExchange [d_date_sk] #3 + ReusedExchange [ca_address_sk,ca_state] #4 InputAdapter BroadcastExchange #8 WholeStageCodegen (5) 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 53dc94532..64de6dbe0 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,99 +1,113 @@ == Physical Plan == -* 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) +* ColumnarToRow (109) ++- CometSort (108) + +- CometColumnarExchange (107) + +- RowToColumnar (106) + +- * Project (105) + +- * BroadcastHashJoin Inner BuildRight (104) + :- * Project (88) + : +- * BroadcastHashJoin Inner BuildRight (87) + : :- * BroadcastHashJoin Inner BuildRight (71) + : : :- * Project (55) + : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : :- * BroadcastHashJoin Inner BuildRight (36) + : : : : :- * HashAggregate (18) + : : : : : +- * ColumnarToRow (17) + : : : : : +- CometColumnarExchange (16) + : : : : : +- RowToColumnar (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 (35) + : : : : +- * HashAggregate (34) + : : : : +- * ColumnarToRow (33) + : : : : +- CometColumnarExchange (32) + : : : : +- RowToColumnar (31) + : : : : +- * HashAggregate (30) + : : : : +- * ColumnarToRow (29) + : : : : +- CometProject (28) + : : : : +- CometBroadcastHashJoin (27) + : : : : :- CometProject (25) + : : : : : +- CometBroadcastHashJoin (24) + : : : : : :- CometFilter (20) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (19) + : : : : : +- CometBroadcastExchange (23) + : : : : : +- CometFilter (22) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (21) + : : : : +- ReusedExchange (26) + : : : +- BroadcastExchange (53) + : : : +- * HashAggregate (52) + : : : +- * ColumnarToRow (51) + : : : +- CometColumnarExchange (50) + : : : +- RowToColumnar (49) + : : : +- * HashAggregate (48) + : : : +- * ColumnarToRow (47) + : : : +- CometProject (46) + : : : +- CometBroadcastHashJoin (45) + : : : :- CometProject (43) + : : : : +- CometBroadcastHashJoin (42) + : : : : :- CometFilter (38) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (37) + : : : : +- CometBroadcastExchange (41) + : : : : +- CometFilter (40) + : : : : +- CometScan parquet spark_catalog.default.date_dim (39) + : : : +- ReusedExchange (44) + : : +- BroadcastExchange (70) + : : +- * HashAggregate (69) + : : +- * ColumnarToRow (68) + : : +- CometColumnarExchange (67) + : : +- RowToColumnar (66) + : : +- * HashAggregate (65) + : : +- * ColumnarToRow (64) + : : +- CometProject (63) + : : +- CometBroadcastHashJoin (62) + : : :- CometProject (60) + : : : +- CometBroadcastHashJoin (59) + : : : :- CometFilter (57) + : : : : +- CometScan parquet spark_catalog.default.web_sales (56) + : : : +- ReusedExchange (58) + : : +- ReusedExchange (61) + : +- BroadcastExchange (86) + : +- * HashAggregate (85) + : +- * ColumnarToRow (84) + : +- CometColumnarExchange (83) + : +- RowToColumnar (82) + : +- * HashAggregate (81) + : +- * ColumnarToRow (80) + : +- CometProject (79) + : +- CometBroadcastHashJoin (78) + : :- CometProject (76) + : : +- CometBroadcastHashJoin (75) + : : :- CometFilter (73) + : : : +- CometScan parquet spark_catalog.default.web_sales (72) + : : +- ReusedExchange (74) + : +- ReusedExchange (77) + +- BroadcastExchange (103) + +- * HashAggregate (102) + +- * ColumnarToRow (101) + +- CometColumnarExchange (100) + +- RowToColumnar (99) + +- * HashAggregate (98) + +- * ColumnarToRow (97) + +- CometProject (96) + +- CometBroadcastHashJoin (95) + :- CometProject (93) + : +- CometBroadcastHashJoin (92) + : :- CometFilter (90) + : : +- CometScan parquet spark_catalog.default.web_sales (89) + : +- ReusedExchange (91) + +- ReusedExchange (94) (1) Scan parquet spark_catalog.default.store_sales @@ -166,18 +180,24 @@ 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] -(15) Exchange +(15) RowToColumnar 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=1] -(16) HashAggregate [codegen id : 12] +(16) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(17) ColumnarToRow [codegen id : 12] +Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] + +(18) 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] -(17) Scan parquet spark_catalog.default.store_sales +(19) 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 [] @@ -185,78 +205,84 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#16), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct -(18) CometFilter +(20) CometFilter Input [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] Condition : isnotnull(ss_addr_sk#14) -(19) Scan parquet spark_catalog.default.date_dim +(21) 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) CometFilter +(22) 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 +(23) 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 +(24) 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 -(23) CometProject +(25) 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] -(24) ReusedExchange [Reuses operator id: 10] +(26) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#21, ca_county#22] -(25) CometBroadcastHashJoin +(27) 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 -(26) CometProject +(28) 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] -(27) ColumnarToRow [codegen id : 2] +(29) 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] +(30) 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] -(29) Exchange +(31) RowToColumnar +Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] + +(32) CometColumnarExchange 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=2] +Arguments: hashpartitioning(ca_county#22, d_qoy#20, d_year#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(30) HashAggregate [codegen id : 3] +(33) ColumnarToRow [codegen id : 3] +Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] + +(34) 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] -(31) BroadcastExchange +(35) BroadcastExchange Input [2]: [ca_county#22, store_sales#25] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] -(32) BroadcastHashJoin [codegen id : 12] +(36) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ca_county#9] Right keys [1]: [ca_county#22] Join type: Inner Join condition: None -(33) Scan parquet spark_catalog.default.store_sales +(37) 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 [] @@ -264,82 +290,88 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#28), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct -(34) CometFilter +(38) CometFilter Input [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] Condition : isnotnull(ss_addr_sk#26) -(35) Scan parquet spark_catalog.default.date_dim +(39) 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 -(36) CometFilter +(40) 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 +(41) CometBroadcastExchange Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] Arguments: [d_date_sk#30, d_year#31, d_qoy#32] -(38) CometBroadcastHashJoin +(42) 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 +(43) 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] -(40) ReusedExchange [Reuses operator id: 10] +(44) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#33, ca_county#34] -(41) CometBroadcastHashJoin +(45) 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 -(42) CometProject +(46) 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] +(47) ColumnarToRow [codegen id : 4] Input [4]: [ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_county#34] -(44) HashAggregate [codegen id : 4] +(48) 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] -(45) Exchange +(49) RowToColumnar +Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] + +(50) CometColumnarExchange 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=4] +Arguments: hashpartitioning(ca_county#34, d_qoy#32, d_year#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(46) HashAggregate [codegen id : 5] +(51) ColumnarToRow [codegen id : 5] +Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] + +(52) 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] -(47) BroadcastExchange +(53) BroadcastExchange Input [2]: [ca_county#34, store_sales#37] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(48) BroadcastHashJoin [codegen id : 12] +(54) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ca_county#22] Right keys [1]: [ca_county#34] Join type: Inner Join condition: None -(49) Project [codegen id : 12] +(55) 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] -(50) Scan parquet spark_catalog.default.web_sales +(56) 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 [] @@ -347,66 +379,72 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(51) CometFilter +(57) 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) -(52) ReusedExchange [Reuses operator id: 5] +(58) ReusedExchange [Reuses operator id: 5] Output [3]: [d_date_sk#42, d_year#43, d_qoy#44] -(53) CometBroadcastHashJoin +(59) 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 -(54) CometProject +(60) 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] -(55) ReusedExchange [Reuses operator id: 10] +(61) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#45, ca_county#46] -(56) CometBroadcastHashJoin +(62) 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 -(57) CometProject +(63) 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] +(64) ColumnarToRow [codegen id : 6] Input [4]: [ws_ext_sales_price#39, d_year#43, d_qoy#44, ca_county#46] -(59) HashAggregate [codegen id : 6] +(65) 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] -(60) Exchange +(66) RowToColumnar +Input [4]: [ca_county#46, d_qoy#44, d_year#43, sum#48] + +(67) CometColumnarExchange 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=6] +Arguments: hashpartitioning(ca_county#46, d_qoy#44, d_year#43, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(61) HashAggregate [codegen id : 7] +(68) ColumnarToRow [codegen id : 7] +Input [4]: [ca_county#46, d_qoy#44, d_year#43, sum#48] + +(69) 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] -(62) BroadcastExchange +(70) BroadcastExchange Input [2]: [ca_county#46, web_sales#50] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] -(63) BroadcastHashJoin [codegen id : 12] +(71) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ca_county#9] Right keys [1]: [ca_county#46] Join type: Inner Join condition: None -(64) Scan parquet spark_catalog.default.web_sales +(72) 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 [] @@ -414,70 +452,76 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(65) CometFilter +(73) 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) -(66) ReusedExchange [Reuses operator id: 21] +(74) ReusedExchange [Reuses operator id: 23] Output [3]: [d_date_sk#55, d_year#56, d_qoy#57] -(67) CometBroadcastHashJoin +(75) 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 -(68) CometProject +(76) 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] -(69) ReusedExchange [Reuses operator id: 10] +(77) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#58, ca_county#59] -(70) CometBroadcastHashJoin +(78) 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 -(71) CometProject +(79) 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] +(80) ColumnarToRow [codegen id : 8] Input [4]: [ws_ext_sales_price#52, d_year#56, d_qoy#57, ca_county#59] -(73) HashAggregate [codegen id : 8] +(81) 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] -(74) Exchange +(82) RowToColumnar +Input [4]: [ca_county#59, d_qoy#57, d_year#56, sum#61] + +(83) CometColumnarExchange 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=8] +Arguments: hashpartitioning(ca_county#59, d_qoy#57, d_year#56, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(75) HashAggregate [codegen id : 9] +(84) ColumnarToRow [codegen id : 9] +Input [4]: [ca_county#59, d_qoy#57, d_year#56, sum#61] + +(85) 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] -(76) BroadcastExchange +(86) BroadcastExchange Input [2]: [ca_county#59, web_sales#62] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=9] -(77) BroadcastHashJoin [codegen id : 12] +(87) 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) -(78) Project [codegen id : 12] +(88) 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] -(79) Scan parquet spark_catalog.default.web_sales +(89) 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 [] @@ -485,158 +529,170 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(80) CometFilter +(90) 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) -(81) ReusedExchange [Reuses operator id: 37] +(91) ReusedExchange [Reuses operator id: 41] Output [3]: [d_date_sk#67, d_year#68, d_qoy#69] -(82) CometBroadcastHashJoin +(92) 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 -(83) CometProject +(93) 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] -(84) ReusedExchange [Reuses operator id: 10] +(94) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#70, ca_county#71] -(85) CometBroadcastHashJoin +(95) 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 -(86) CometProject +(96) 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] -(87) ColumnarToRow [codegen id : 10] +(97) 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] +(98) 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] -(89) Exchange +(99) RowToColumnar +Input [4]: [ca_county#71, d_qoy#69, d_year#68, sum#73] + +(100) CometColumnarExchange 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=10] +Arguments: hashpartitioning(ca_county#71, d_qoy#69, d_year#68, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(90) HashAggregate [codegen id : 11] +(101) ColumnarToRow [codegen id : 11] +Input [4]: [ca_county#71, d_qoy#69, d_year#68, sum#73] + +(102) 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] -(91) BroadcastExchange +(103) BroadcastExchange Input [2]: [ca_county#71, web_sales#74] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(92) BroadcastHashJoin [codegen id : 12] +(104) 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) -(93) Project [codegen id : 12] +(105) 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] -(94) Exchange +(106) RowToColumnar +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] + +(107) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=12] + +(108) CometSort 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=12] +Arguments: [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], [ca_county#9 ASC NULLS FIRST] -(95) Sort [codegen id : 13] +(109) ColumnarToRow [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 (99) -+- * ColumnarToRow (98) - +- CometFilter (97) - +- CometScan parquet spark_catalog.default.date_dim (96) +BroadcastExchange (113) ++- * ColumnarToRow (112) + +- CometFilter (111) + +- CometScan parquet spark_catalog.default.date_dim (110) -(96) Scan parquet spark_catalog.default.date_dim +(110) 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 -(97) CometFilter +(111) 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)) -(98) ColumnarToRow [codegen id : 1] +(112) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -(99) BroadcastExchange +(113) 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=13] -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) +Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#16 IN dynamicpruning#17 +BroadcastExchange (117) ++- * ColumnarToRow (116) + +- CometFilter (115) + +- CometScan parquet spark_catalog.default.date_dim (114) -(100) Scan parquet spark_catalog.default.date_dim +(114) 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 -(101) CometFilter +(115) 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)) -(102) ColumnarToRow [codegen id : 1] +(116) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] -(103) BroadcastExchange +(117) 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=14] -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) +Subquery:3 Hosting operator id = 37 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#29 +BroadcastExchange (121) ++- * ColumnarToRow (120) + +- CometFilter (119) + +- CometScan parquet spark_catalog.default.date_dim (118) -(104) Scan parquet spark_catalog.default.date_dim +(118) 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 -(105) CometFilter +(119) 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)) -(106) ColumnarToRow [codegen id : 1] +(120) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] -(107) BroadcastExchange +(121) 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=15] -Subquery:4 Hosting operator id = 50 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#4 +Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#4 -Subquery:5 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#17 +Subquery:5 Hosting operator id = 72 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#17 -Subquery:6 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#65 IN dynamicpruning#29 +Subquery:6 Hosting operator id = 89 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 9040179da..4567f8e73 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,150 +1,164 @@ WholeStageCodegen (13) - Sort [ca_county] + ColumnarToRow InputAdapter - Exchange [ca_county] #1 - 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] + CometSort [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] + CometColumnarExchange [ca_county] #1 + RowToColumnar + 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] - BroadcastHashJoin [ca_county,ca_county] - Project [ca_county,d_year,store_sales,store_sales,store_sales] + Project [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] + BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] BroadcastHashJoin [ca_county,ca_county] - BroadcastHashJoin [ca_county,ca_county] - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] + Project [ca_county,d_year,store_sales,store_sales,store_sales] + BroadcastHashJoin [ca_county,ca_county] + BroadcastHashJoin [ca_county,ca_county] + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [ca_county,d_qoy,d_year] #2 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #4 + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [ca_address_sk,ca_county] #5 + CometFilter [ca_address_sk,ca_county] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [ca_county,d_qoy,d_year] #7 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_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 #8 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #9 + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter - Exchange [ca_county,d_qoy,d_year] #2 - WholeStageCodegen (1) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] + BroadcastExchange #10 + WholeStageCodegen (5) + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] ColumnarToRow InputAdapter - CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] - CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + CometColumnarExchange [ca_county,d_qoy,d_year] #11 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_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 #12 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #13 CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #4 - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [ca_address_sk,ca_county] #5 - CometFilter [ca_address_sk,ca_county] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] + ReusedExchange [ca_address_sk,ca_county] #5 + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (7) + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] + ColumnarToRow InputAdapter - Exchange [ca_county,d_qoy,d_year] #7 - WholeStageCodegen (2) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] + CometColumnarExchange [ca_county,d_qoy,d_year] #15 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 + ReusedExchange [d_date_sk,d_year,d_qoy] #4 + ReusedExchange [ca_address_sk,ca_county] #5 + InputAdapter + BroadcastExchange #16 + WholeStageCodegen (9) + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [ca_county,d_qoy,d_year] #17 + RowToColumnar + WholeStageCodegen (8) + HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] ColumnarToRow InputAdapter - CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] - CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_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 #8 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #9 - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 + ReusedExchange [d_date_sk,d_year,d_qoy] #9 ReusedExchange [ca_address_sk,ca_county] #5 - InputAdapter - 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] #11 - WholeStageCodegen (4) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] - CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_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 #12 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #13 - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - ReusedExchange [ca_address_sk,ca_county] #5 - InputAdapter - 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] #15 - WholeStageCodegen (6) - HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] - CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 - ReusedExchange [d_date_sk,d_year,d_qoy] #4 - ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter - BroadcastExchange #16 - WholeStageCodegen (9) + 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] #17 - WholeStageCodegen (8) - HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] - CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 - ReusedExchange [d_date_sk,d_year,d_qoy] #9 - ReusedExchange [ca_address_sk,ca_county] #5 - InputAdapter - 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] #19 - WholeStageCodegen (10) - HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] - CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 - ReusedExchange [d_date_sk,d_year,d_qoy] #13 - ReusedExchange [ca_address_sk,ca_county] #5 + ColumnarToRow + InputAdapter + CometColumnarExchange [ca_county,d_qoy,d_year] #19 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 + 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 fe9e7d49a..6a4fcb1d8 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,35 +1,39 @@ == Physical Plan == -* 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) +* HashAggregate (35) ++- * ColumnarToRow (34) + +- CometColumnarExchange (33) + +- RowToColumnar (32) + +- * HashAggregate (31) + +- * Project (30) + +- * BroadcastHashJoin Inner BuildRight (29) + :- * Project (27) + : +- * BroadcastHashJoin Inner BuildRight (26) + : :- * 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 (25) + : +- * Filter (24) + : +- * HashAggregate (23) + : +- * ColumnarToRow (22) + : +- CometColumnarExchange (21) + : +- RowToColumnar (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 (28) (1) Scan parquet spark_catalog.default.catalog_sales @@ -125,60 +129,72 @@ Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#8))] Aggregate Attributes [2]: [sum#13, count#14] Results [3]: [cs_item_sk#7, sum#15, count#16] -(20) Exchange +(20) RowToColumnar Input [3]: [cs_item_sk#7, sum#15, count#16] -Arguments: hashpartitioning(cs_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(21) HashAggregate [codegen id : 2] +(21) CometColumnarExchange +Input [3]: [cs_item_sk#7, sum#15, count#16] +Arguments: hashpartitioning(cs_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(22) ColumnarToRow [codegen id : 2] +Input [3]: [cs_item_sk#7, sum#15, count#16] + +(23) 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))#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] -(22) Filter [codegen id : 2] +(24) 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) -(23) BroadcastExchange +(25) 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] -(24) BroadcastHashJoin [codegen id : 4] +(26) 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))#18) -(25) Project [codegen id : 4] +(27) 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))#18, cs_item_sk#7] -(26) ReusedExchange [Reuses operator id: 36] +(28) ReusedExchange [Reuses operator id: 40] Output [1]: [d_date_sk#19] -(27) BroadcastHashJoin [codegen id : 4] +(29) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#3] Right keys [1]: [d_date_sk#19] Join type: Inner Join condition: None -(28) Project [codegen id : 4] +(30) 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#19] -(29) HashAggregate [codegen id : 4] +(31) 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#20] Results [1]: [sum#21] -(30) Exchange +(32) RowToColumnar +Input [1]: [sum#21] + +(33) CometColumnarExchange +Input [1]: [sum#21] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(34) ColumnarToRow [codegen id : 5] Input [1]: [sum#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] -(31) HashAggregate [codegen id : 5] +(35) HashAggregate [codegen id : 5] Input [1]: [sum#21] Keys: [] Functions [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))] @@ -188,32 +204,32 @@ Results [1]: [MakeDecimal(sum(UnscaledValue(cs_ext_discount_amt#2))#22,17,2) AS ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (36) -+- * ColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan parquet spark_catalog.default.date_dim (32) +BroadcastExchange (40) ++- * ColumnarToRow (39) + +- CometProject (38) + +- CometFilter (37) + +- CometScan parquet spark_catalog.default.date_dim (36) -(32) Scan parquet spark_catalog.default.date_dim +(36) 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 -(33) CometFilter +(37) 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)) -(34) CometProject +(38) CometProject Input [2]: [d_date_sk#19, d_date#24] Arguments: [d_date_sk#19], [d_date_sk#19] -(35) ColumnarToRow [codegen id : 1] +(39) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] -(36) BroadcastExchange +(40) BroadcastExchange Input [1]: [d_date_sk#19] 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/q32/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/simplified.txt index b6ba1fbab..d415c4b58 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,50 +1,54 @@ WholeStageCodegen (5) HashAggregate [sum] [sum(UnscaledValue(cs_ext_discount_amt)),excess discount amount,sum] - InputAdapter - Exchange #1 - 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))] - ColumnarToRow - InputAdapter - CometProject [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] - CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] - CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk] #3 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + ColumnarToRow + InputAdapter + CometColumnarExchange #1 + RowToColumnar + 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))] + ColumnarToRow + InputAdapter + CometProject [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] + CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk] #3 + CometProject [i_item_sk] + CometFilter [i_item_sk,i_manufact_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + InputAdapter + BroadcastExchange #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] + ColumnarToRow + InputAdapter + CometColumnarExchange [cs_item_sk] #5 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [cs_item_sk,cs_ext_discount_amt] [sum,count,sum,count] + ColumnarToRow + InputAdapter + CometProject [cs_item_sk,cs_ext_discount_amt] + CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,d_date_sk] + CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #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 (1) - HashAggregate [cs_item_sk,cs_ext_discount_amt] [sum,count,sum,count] - ColumnarToRow - InputAdapter - CometProject [cs_item_sk,cs_ext_discount_amt] - CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,d_date_sk] - CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - ReusedExchange [d_date_sk] #2 + 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 4c00423c0..d4e67403e 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 @@ -1,67 +1,75 @@ == Physical Plan == -TakeOrderedAndProject (63) -+- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- Union (59) - :- * HashAggregate (28) - : +- Exchange (27) - : +- * HashAggregate (26) - : +- * 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) - : +- * 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) - +- * 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) +TakeOrderedAndProject (71) ++- * HashAggregate (70) + +- * ColumnarToRow (69) + +- CometColumnarExchange (68) + +- RowToColumnar (67) + +- * HashAggregate (66) + +- Union (65) + :- * HashAggregate (30) + : +- * ColumnarToRow (29) + : +- CometColumnarExchange (28) + : +- RowToColumnar (27) + : +- * HashAggregate (26) + : +- * 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 (47) + : +- * ColumnarToRow (46) + : +- CometColumnarExchange (45) + : +- RowToColumnar (44) + : +- * HashAggregate (43) + : +- * ColumnarToRow (42) + : +- CometProject (41) + : +- CometBroadcastHashJoin (40) + : :- CometProject (38) + : : +- CometBroadcastHashJoin (37) + : : :- CometProject (35) + : : : +- CometBroadcastHashJoin (34) + : : : :- CometFilter (32) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (31) + : : : +- ReusedExchange (33) + : : +- ReusedExchange (36) + : +- ReusedExchange (39) + +- * HashAggregate (64) + +- * ColumnarToRow (63) + +- CometColumnarExchange (62) + +- RowToColumnar (61) + +- * HashAggregate (60) + +- * ColumnarToRow (59) + +- CometProject (58) + +- CometBroadcastHashJoin (57) + :- CometProject (55) + : +- CometBroadcastHashJoin (54) + : :- CometProject (52) + : : +- CometBroadcastHashJoin (51) + : : :- CometFilter (49) + : : : +- CometScan parquet spark_catalog.default.web_sales (48) + : : +- ReusedExchange (50) + : +- ReusedExchange (53) + +- ReusedExchange (56) (1) Scan parquet spark_catalog.default.store_sales @@ -190,18 +198,24 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] Aggregate Attributes [1]: [sum#15] Results [2]: [i_manufact_id#12, sum#16] -(27) Exchange +(27) RowToColumnar Input [2]: [i_manufact_id#12, sum#16] -Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(28) HashAggregate [codegen id : 2] +(28) CometColumnarExchange +Input [2]: [i_manufact_id#12, sum#16] +Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(29) ColumnarToRow [codegen id : 2] +Input [2]: [i_manufact_id#12, sum#16] + +(30) 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))#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 +(31) Scan parquet spark_catalog.default.catalog_sales 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 [] @@ -209,68 +223,74 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#22), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(30) CometFilter +(32) CometFilter 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] +(33) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#24] -(32) CometBroadcastHashJoin +(34) 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 +(35) 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] +(36) ReusedExchange [Reuses operator id: 12] Output [1]: [ca_address_sk#25] -(35) CometBroadcastHashJoin +(37) 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 +(38) 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] +(39) ReusedExchange [Reuses operator id: 22] Output [2]: [i_item_sk#26, i_manufact_id#27] -(38) CometBroadcastHashJoin +(40) 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 +(41) 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] +(42) ColumnarToRow [codegen id : 3] Input [2]: [cs_ext_sales_price#21, i_manufact_id#27] -(41) HashAggregate [codegen id : 3] +(43) 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 +(44) RowToColumnar +Input [2]: [i_manufact_id#27, sum#29] + +(45) CometColumnarExchange +Input [2]: [i_manufact_id#27, sum#29] +Arguments: hashpartitioning(i_manufact_id#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(46) ColumnarToRow [codegen id : 4] Input [2]: [i_manufact_id#27, sum#29] -Arguments: hashpartitioning(i_manufact_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(43) HashAggregate [codegen id : 4] +(47) 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 +(48) Scan parquet spark_catalog.default.web_sales 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 [] @@ -278,125 +298,137 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(45) CometFilter +(49) CometFilter 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] +(50) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#37] -(47) CometBroadcastHashJoin +(51) 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 +(52) 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] +(53) ReusedExchange [Reuses operator id: 12] Output [1]: [ca_address_sk#38] -(50) CometBroadcastHashJoin +(54) 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 +(55) 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] +(56) ReusedExchange [Reuses operator id: 22] Output [2]: [i_item_sk#39, i_manufact_id#40] -(53) CometBroadcastHashJoin +(57) 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 +(58) 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] +(59) ColumnarToRow [codegen id : 5] Input [2]: [ws_ext_sales_price#34, i_manufact_id#40] -(56) HashAggregate [codegen id : 5] +(60) 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 +(61) RowToColumnar Input [2]: [i_manufact_id#40, sum#42] -Arguments: hashpartitioning(i_manufact_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(58) HashAggregate [codegen id : 6] +(62) CometColumnarExchange +Input [2]: [i_manufact_id#40, sum#42] +Arguments: hashpartitioning(i_manufact_id#40, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(63) ColumnarToRow [codegen id : 6] +Input [2]: [i_manufact_id#40, sum#42] + +(64) 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 +(65) Union -(60) HashAggregate [codegen id : 7] +(66) 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 +(67) RowToColumnar +Input [3]: [i_manufact_id#12, sum#47, isEmpty#48] + +(68) CometColumnarExchange +Input [3]: [i_manufact_id#12, sum#47, isEmpty#48] +Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(69) ColumnarToRow [codegen id : 8] 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 : 8] +(70) 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 +(71) TakeOrderedAndProject 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 ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (68) -+- * ColumnarToRow (67) - +- CometProject (66) - +- CometFilter (65) - +- CometScan parquet spark_catalog.default.date_dim (64) +BroadcastExchange (76) ++- * ColumnarToRow (75) + +- CometProject (74) + +- CometFilter (73) + +- CometScan parquet spark_catalog.default.date_dim (72) -(64) Scan parquet spark_catalog.default.date_dim +(72) Scan parquet spark_catalog.default.date_dim 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 +(73) 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)) -(66) CometProject +(74) CometProject 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] +(75) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(68) BroadcastExchange +(76) BroadcastExchange Input [1]: [d_date_sk#6] 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#22 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 31 Hosting Expression = cs_sold_date_sk#22 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 48 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 bf0ef7ae4..7680cab6c 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,89 +1,97 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (8) HashAggregate [i_manufact_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - InputAdapter - Exchange [i_manufact_id] #1 - WholeStageCodegen (7) - HashAggregate [i_manufact_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (2) - HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_manufact_id] #2 - WholeStageCodegen (1) - HashAggregate [i_manufact_id,ss_ext_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [ss_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_manufact_id] - CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ca_address_sk] #5 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk,i_manufact_id] #6 - CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id] - CometFilter [i_item_sk,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange [i_manufact_id] #7 - CometProject [i_manufact_id] - CometFilter [i_category,i_manufact_id] - 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] #8 - WholeStageCodegen (3) - HashAggregate [i_manufact_id,cs_ext_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [cs_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_manufact_id] - CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] - CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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 - 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] #9 - WholeStageCodegen (5) - HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [ws_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_manufact_id] - CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] - CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_manufact_id] #6 + ColumnarToRow + InputAdapter + CometColumnarExchange [i_manufact_id] #1 + RowToColumnar + WholeStageCodegen (7) + HashAggregate [i_manufact_id,total_sales] [sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (2) + HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_manufact_id] #2 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [i_manufact_id,ss_ext_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [ss_ext_sales_price,i_manufact_id] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_manufact_id] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ca_address_sk] #5 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometBroadcastExchange [i_item_sk,i_manufact_id] #6 + CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id] + CometFilter [i_item_sk,i_manufact_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + CometBroadcastExchange [i_manufact_id] #7 + CometProject [i_manufact_id] + CometFilter [i_category,i_manufact_id] + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_manufact_id] #8 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [i_manufact_id,cs_ext_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [cs_ext_sales_price,i_manufact_id] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_manufact_id] + CometProject [cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] + CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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 + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_manufact_id] #9 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [ws_ext_sales_price,i_manufact_id] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_manufact_id] + CometProject [ws_item_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] + CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 + 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 5609dc11c..c33735fa4 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,37 +1,36 @@ == Physical Plan == -* 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) +* ColumnarToRow (32) ++- CometSort (31) + +- CometColumnarExchange (30) + +- CometProject (29) + +- CometBroadcastHashJoin (28) + :- CometFilter (24) + : +- CometHashAggregate (23) + : +- CometColumnarExchange (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) + +- CometBroadcastExchange (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.customer (25) (1) Scan parquet spark_catalog.default.store_sales @@ -135,90 +134,84 @@ 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)] -(22) ColumnarToRow [codegen id : 1] +(22) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(24) HashAggregate [codegen id : 3] +(23) CometHashAggregate 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)#17] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#17 AS cnt#18] -(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)) +(24) CometFilter +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] +Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) -(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] +(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] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(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) +(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) -(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] +(27) CometBroadcastExchange +Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Arguments: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -(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] +(28) CometBroadcastHashJoin +Left output [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] +Right output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight -(30) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#19] -Join type: Inner -Join condition: None +(29) CometProject +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] +Arguments: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17], [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -(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] +(30) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] -(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] +(31) CometSort +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, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17], [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] -(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 +(32) ColumnarToRow [codegen id : 1] +Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (38) -+- * ColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.date_dim (34) +BroadcastExchange (37) ++- * ColumnarToRow (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) -(34) Scan parquet spark_catalog.default.date_dim +(33) 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 -(35) CometFilter +(34) 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)) -(36) CometProject +(35) CometProject Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(37) ColumnarToRow [codegen id : 1] +(36) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(38) BroadcastExchange +(37) BroadcastExchange Input [1]: [d_date_sk#7] -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/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt index e1f7f9bd2..c131bd9c0 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,50 +1,42 @@ -WholeStageCodegen (4) - Sort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,count] - CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] - CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] - CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #5 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange [hd_demo_sk] #6 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 + CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometFilter [ss_ticket_number,ss_customer_sk,cnt] + CometHashAggregate [ss_ticket_number,ss_customer_sk,cnt,count,count(1)] + CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] + CometProject [ss_customer_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange [s_store_sk] #5 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_county] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county] + CometBroadcastExchange [hd_demo_sk] #6 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] #7 + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] 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 c06c1dd16..b35338d18 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,48 +1,50 @@ == Physical Plan == -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) +TakeOrderedAndProject (46) ++- * HashAggregate (45) + +- * ColumnarToRow (44) + +- CometColumnarExchange (43) + +- RowToColumnar (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 @@ -238,50 +240,56 @@ Functions [10]: [partial_count(1), partial_min(cd_dep_count#25), partial_max(cd_ 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] -(42) Exchange +(42) RowToColumnar 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] -(43) HashAggregate [codegen id : 6] +(43) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=5] + +(44) ColumnarToRow [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] + +(45) 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] -(44) TakeOrderedAndProject +(46) 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 = 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) +BroadcastExchange (51) ++- * ColumnarToRow (50) + +- CometProject (49) + +- CometFilter (48) + +- CometScan parquet spark_catalog.default.date_dim (47) -(45) Scan parquet spark_catalog.default.date_dim +(47) 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 -(46) CometFilter +(48) 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)) -(47) CometProject +(49) CometProject Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(48) ColumnarToRow [codegen id : 1] +(50) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(49) BroadcastExchange +(51) BroadcastExchange Input [1]: [d_date_sk#9] 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/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/simplified.txt index 9deb51342..f9ef660d5 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,70 +1,72 @@ 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 (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 (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] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] - CometFilter [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] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,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 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + RowToColumnar + 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] + Project [c_current_cdemo_sk,ca_state] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] + Filter [exists,exists] + BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,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 + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [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] + CometBroadcastExchange [ss_customer_sk] #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,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 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [cs_ship_customer_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,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 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) + BroadcastExchange #8 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - 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] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + 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-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 1fdba8689..ca46904b3 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,33 +1,37 @@ == Physical Plan == -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) +TakeOrderedAndProject (33) ++- * Project (32) + +- Window (31) + +- * ColumnarToRow (30) + +- CometSort (29) + +- CometColumnarExchange (28) + +- RowToColumnar (27) + +- * HashAggregate (26) + +- * ColumnarToRow (25) + +- CometColumnarExchange (24) + +- RowToColumnar (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 @@ -136,66 +140,78 @@ Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(Unscale Aggregate Attributes [2]: [sum#17, sum#18] Results [5]: [i_category#14, i_class#15, spark_grouping_id#16, sum#19, sum#20] -(23) Exchange +(23) RowToColumnar 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] -(24) HashAggregate [codegen id : 2] +(24) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(25) ColumnarToRow [codegen id : 2] +Input [5]: [i_category#14, i_class#15, spark_grouping_id#16, sum#19, sum#20] + +(26) 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))#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] -(25) Exchange +(27) RowToColumnar +Input [7]: [gross_margin#23, i_category#14, i_class#15, lochierarchy#24, _w0#25, _w1#26, _w2#27] + +(28) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] + +(29) CometSort 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] +Arguments: [gross_margin#23, i_category#14, i_class#15, lochierarchy#24, _w0#25, _w1#26, _w2#27], [_w1#26 ASC NULLS FIRST, _w2#27 ASC NULLS FIRST, _w0#25 ASC NULLS FIRST] -(26) Sort [codegen id : 3] +(30) ColumnarToRow [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 -(27) Window +(31) 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] -(28) Project [codegen id : 4] +(32) 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] -(29) TakeOrderedAndProject +(33) 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 (34) -+- * ColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan parquet spark_catalog.default.date_dim (30) +BroadcastExchange (38) ++- * ColumnarToRow (37) + +- CometProject (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.date_dim (34) -(30) Scan parquet spark_catalog.default.date_dim +(34) 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 -(31) CometFilter +(35) 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)) -(32) CometProject +(36) CometProject Input [2]: [d_date_sk#7, d_year#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(33) ColumnarToRow [codegen id : 1] +(37) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(34) BroadcastExchange +(38) BroadcastExchange Input [1]: [d_date_sk#7] 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 58d617430..f3febe692 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 @@ -4,42 +4,46 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i InputAdapter Window [_w0,_w1,_w2] WholeStageCodegen (3) - Sort [_w1,_w2,_w0] + ColumnarToRow InputAdapter - Exchange [_w1,_w2] #1 - 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 (1) - HashAggregate [i_category,i_class,spark_grouping_id,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] - 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,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] - CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] - CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometFilter [i_item_sk,i_class,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #6 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] + CometColumnarExchange [_w1,_w2] #1 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class,spark_grouping_id] #2 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [i_category,i_class,spark_grouping_id,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] + 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,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] + CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometFilter [i_item_sk,i_class,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometBroadcastExchange [s_store_sk] #6 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + 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 6eb0f14b1..42dab0b38 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,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (26) -+- * HashAggregate (25) - +- Exchange (24) - +- * ColumnarToRow (23) +* ColumnarToRow (26) ++- CometTakeOrderedAndProject (25) + +- CometHashAggregate (24) + +- CometColumnarExchange (23) +- CometHashAggregate (22) +- CometProject (21) +- CometBroadcastHashJoin (20) @@ -132,23 +132,21 @@ 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: [] -(23) ColumnarToRow [codegen id : 1] +(23) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(24) 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] -(26) TakeOrderedAndProject +(25) CometTakeOrderedAndProject +Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#2 ASC NULLS FIRST], output=[i_item_id#2,i_item_desc#3,i_current_price#4]), [i_item_id#2, i_item_desc#3, i_current_price#4], 100, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4] + +(26) ColumnarToRow [codegen id : 1] 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 ===== 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 efd86d5ab..39edfb519 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,38 +1,36 @@ -TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - 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 (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,i_item_id,i_item_desc,i_current_price,cs_item_sk] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2 - CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,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,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] - CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 - CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_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_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] + CometHashAggregate [i_item_id,i_item_desc,i_current_price] + CometColumnarExchange [i_item_id,i_item_desc,i_current_price] #1 + CometHashAggregate [i_item_id,i_item_desc,i_current_price] + CometProject [i_item_id,i_item_desc,i_current_price] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,cs_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2 + CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,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,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] + CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] + CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 + CometProject [inv_item_sk,inv_date_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_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_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [cs_item_sk] + CometFilter [cs_item_sk,cs_sold_date_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 b6b11827d..091883bd6 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,53 +1,51 @@ == Physical Plan == -* 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) +* ColumnarToRow (47) ++- CometHashAggregate (46) + +- CometColumnarExchange (45) + +- CometHashAggregate (44) + +- CometProject (43) + +- CometBroadcastHashJoin (42) + :- CometBroadcastHashJoin (29) + : :- CometHashAggregate (16) + : : +- CometColumnarExchange (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) + : +- CometBroadcastExchange (28) + : +- CometHashAggregate (27) + : +- CometColumnarExchange (26) + : +- CometHashAggregate (25) + : +- CometProject (24) + : +- CometBroadcastHashJoin (23) + : :- CometProject (21) + : : +- CometBroadcastHashJoin (20) + : : :- CometFilter (18) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (17) + : : +- ReusedExchange (19) + : +- ReusedExchange (22) + +- CometBroadcastExchange (41) + +- CometHashAggregate (40) + +- CometColumnarExchange (39) + +- CometHashAggregate (38) + +- CometProject (37) + +- CometBroadcastHashJoin (36) + :- CometProject (34) + : +- CometBroadcastHashJoin (33) + : :- CometFilter (31) + : : +- CometScan parquet spark_catalog.default.web_sales (30) + : +- ReusedExchange (32) + +- ReusedExchange (35) (1) Scan parquet spark_catalog.default.store_sales @@ -119,21 +117,16 @@ 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: [] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(17) HashAggregate [codegen id : 6] +(16) 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#9, c_first_name#8, d_date#5] -(18) Scan parquet spark_catalog.default.catalog_sales +(17) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_bill_customer_sk#10, cs_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] @@ -141,64 +134,58 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#11), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(19) CometFilter +(18) CometFilter Input [2]: [cs_bill_customer_sk#10, cs_sold_date_sk#11] Condition : isnotnull(cs_bill_customer_sk#10) -(20) ReusedExchange [Reuses operator id: 6] +(19) ReusedExchange [Reuses operator id: 6] Output [2]: [d_date_sk#13, d_date#14] -(21) CometBroadcastHashJoin +(20) 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 -(22) CometProject +(21) 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] -(23) ReusedExchange [Reuses operator id: 11] +(22) ReusedExchange [Reuses operator id: 11] Output [3]: [c_customer_sk#15, c_first_name#16, c_last_name#17] -(24) CometBroadcastHashJoin +(23) 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 -(25) CometProject +(24) 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] -(26) CometHashAggregate +(25) 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: [] -(27) ColumnarToRow [codegen id : 2] +(26) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=2] -(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] +(27) 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#17, c_first_name#16, d_date#14] -(30) BroadcastExchange +(28) CometBroadcastExchange 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] +Arguments: [c_last_name#17, c_first_name#16, d_date#14] -(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 +(29) CometBroadcastHashJoin +Left output [3]: [c_last_name#9, c_first_name#8, d_date#5] +Right output [3]: [c_last_name#17, c_first_name#16, d_date#14] +Arguments: [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)], [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)], LeftSemi, BuildRight -(32) Scan parquet spark_catalog.default.web_sales +(30) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] @@ -206,119 +193,111 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#19), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(33) CometFilter +(31) CometFilter Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] Condition : isnotnull(ws_bill_customer_sk#18) -(34) ReusedExchange [Reuses operator id: 6] +(32) ReusedExchange [Reuses operator id: 6] Output [2]: [d_date_sk#21, d_date#22] -(35) CometBroadcastHashJoin +(33) 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 -(36) CometProject +(34) 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] -(37) ReusedExchange [Reuses operator id: 11] +(35) ReusedExchange [Reuses operator id: 11] Output [3]: [c_customer_sk#23, c_first_name#24, c_last_name#25] -(38) CometBroadcastHashJoin +(36) 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 -(39) CometProject +(37) 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] -(40) CometHashAggregate +(38) 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: [] -(41) ColumnarToRow [codegen id : 4] +(39) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=3] -(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] - -(43) HashAggregate [codegen id : 5] +(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#25, c_first_name#24, d_date#22] -(44) BroadcastExchange +(41) CometBroadcastExchange 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] +Arguments: [c_last_name#25, c_first_name#24, d_date#22] -(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 +(42) CometBroadcastHashJoin +Left output [3]: [c_last_name#9, c_first_name#8, d_date#5] +Right output [3]: [c_last_name#25, c_first_name#24, d_date#22] +Arguments: [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)], [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)], LeftSemi, BuildRight -(46) Project [codegen id : 6] -Output: [] +(43) CometProject Input [3]: [c_last_name#9, c_first_name#8, d_date#5] -(47) HashAggregate [codegen id : 6] +(44) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#26] -Results [1]: [count#27] -(48) Exchange -Input [1]: [count#27] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +(45) CometColumnarExchange +Input [1]: [count#26] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(49) HashAggregate [codegen id : 7] -Input [1]: [count#27] +(46) CometHashAggregate +Input [1]: [count#26] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#28] -Results [1]: [count(1)#28 AS count(1)#29] + +(47) ColumnarToRow [codegen id : 1] +Input [1]: [count(1)#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 -BroadcastExchange (54) -+- * ColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan parquet spark_catalog.default.date_dim (50) +BroadcastExchange (52) ++- * ColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) -(50) Scan parquet spark_catalog.default.date_dim +(48) 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 -(51) CometFilter +(49) 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)) -(52) CometProject +(50) 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] -(53) ColumnarToRow [codegen id : 1] +(51) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#4, d_date#5] -(54) BroadcastExchange +(52) BroadcastExchange Input [2]: [d_date_sk#4, d_date#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=5] -Subquery:2 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#11 IN dynamicpruning#3 +Subquery:2 Hosting operator id = 17 Hosting Expression = cs_sold_date_sk#11 IN dynamicpruning#3 -Subquery:3 Hosting operator id = 32 Hosting Expression = ws_sold_date_sk#19 IN dynamicpruning#3 +Subquery:3 Hosting operator id = 30 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 9d667265c..77970e8d3 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,75 +1,59 @@ -WholeStageCodegen (7) - HashAggregate [count] [count(1),count(1),count] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (6) - HashAggregate [count,count] - Project - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #2 - 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,d_date,c_customer_sk,c_first_name,c_last_name] - CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_customer_sk,ss_sold_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,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #4 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,d_date] - 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,d_date,c_customer_sk,c_first_name,c_last_name] - CometProject [cs_bill_customer_sk,d_date] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] - CometFilter [cs_bill_customer_sk,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_date] #4 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - HashAggregate [c_last_name,c_first_name,d_date] - 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,d_date,c_customer_sk,c_first_name,c_last_name] - CometProject [ws_bill_customer_sk,d_date] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_bill_customer_sk,ws_sold_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,d_date] #4 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 + CometHashAggregate [count(1),count,count(1)] + CometColumnarExchange #1 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #2 + CometHashAggregate [c_last_name,c_first_name,d_date] + CometProject [c_last_name,c_first_name,d_date] + CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] + CometProject [ss_customer_sk,d_date] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_customer_sk,ss_sold_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,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange [d_date_sk,d_date] #4 + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 + CometFilter [c_customer_sk,c_first_name,c_last_name] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] + CometBroadcastExchange [c_last_name,c_first_name,d_date] #6 + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #7 + CometHashAggregate [c_last_name,c_first_name,d_date] + CometProject [c_last_name,c_first_name,d_date] + CometBroadcastHashJoin [cs_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] + CometProject [cs_bill_customer_sk,d_date] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] + CometFilter [cs_bill_customer_sk,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_date] #4 + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 + CometBroadcastExchange [c_last_name,c_first_name,d_date] #8 + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #9 + CometHashAggregate [c_last_name,c_first_name,d_date] + CometProject [c_last_name,c_first_name,d_date] + CometBroadcastHashJoin [ws_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] + CometProject [ws_bill_customer_sk,d_date] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_bill_customer_sk,ws_sold_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,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 5d6b73e88..e266e3e1c 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,52 +1,58 @@ == Physical Plan == -* 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) +* ColumnarToRow (54) ++- CometSort (53) + +- CometColumnarExchange (52) + +- RowToColumnar (51) + +- * BroadcastHashJoin Inner BuildRight (50) + :- * Project (26) + : +- * Filter (25) + : +- * HashAggregate (24) + : +- * ColumnarToRow (23) + : +- CometColumnarExchange (22) + : +- RowToColumnar (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 (49) + +- * Project (48) + +- * Filter (47) + +- * HashAggregate (46) + +- * ColumnarToRow (45) + +- CometColumnarExchange (44) + +- RowToColumnar (43) + +- * HashAggregate (42) + +- * ColumnarToRow (41) + +- CometProject (40) + +- CometBroadcastHashJoin (39) + :- CometProject (34) + : +- CometBroadcastHashJoin (33) + : :- CometProject (31) + : : +- CometBroadcastHashJoin (30) + : : :- CometFilter (28) + : : : +- CometScan parquet spark_catalog.default.inventory (27) + : : +- ReusedExchange (29) + : +- ReusedExchange (32) + +- CometBroadcastExchange (38) + +- CometProject (37) + +- CometFilter (36) + +- CometScan parquet spark_catalog.default.date_dim (35) (1) Scan parquet spark_catalog.default.inventory @@ -147,26 +153,32 @@ Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), par 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] -(21) Exchange +(21) RowToColumnar 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] -(22) HashAggregate [codegen id : 4] +(22) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(23) ColumnarToRow [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] + +(24) 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))#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] -(23) Filter [codegen id : 4] +(25) 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 -(24) Project [codegen id : 4] +(26) 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] -(25) Scan parquet spark_catalog.default.inventory +(27) 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 [] @@ -174,168 +186,180 @@ PartitionFilters: [isnotnull(inv_date_sk#30), dynamicpruningexpression(inv_date_ PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(26) CometFilter +(28) 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)) -(27) ReusedExchange [Reuses operator id: 5] +(29) ReusedExchange [Reuses operator id: 5] Output [1]: [i_item_sk#32] -(28) CometBroadcastHashJoin +(30) 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 -(29) CometProject +(31) 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] -(30) ReusedExchange [Reuses operator id: 10] +(32) ReusedExchange [Reuses operator id: 10] Output [2]: [w_warehouse_sk#33, w_warehouse_name#34] -(31) CometBroadcastHashJoin +(33) 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 -(32) CometProject +(34) 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] -(33) Scan parquet spark_catalog.default.date_dim +(35) 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 -(34) CometFilter +(36) 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 +(37) 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 +(38) CometBroadcastExchange Input [2]: [d_date_sk#35, d_moy#37] Arguments: [d_date_sk#35, d_moy#37] -(37) CometBroadcastHashJoin +(39) 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 +(40) 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] +(41) 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] +(42) 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 +(43) RowToColumnar +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] + +(44) CometColumnarExchange 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] +Arguments: hashpartitioning(w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(42) HashAggregate [codegen id : 3] +(45) ColumnarToRow [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] + +(46) 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] +(47) 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] +(48) 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 +(49) 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] +(50) BroadcastHashJoin [codegen id : 4] Left keys [2]: [i_item_sk#6, w_warehouse_sk#7] Right keys [2]: [i_item_sk#32, w_warehouse_sk#33] Join type: Inner Join condition: None -(47) Exchange +(51) RowToColumnar +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] + +(52) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=4] + +(53) CometSort 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] +Arguments: [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], [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] -(48) Sort [codegen id : 5] +(54) ColumnarToRow [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 (53) -+- * ColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometScan parquet spark_catalog.default.date_dim (49) +BroadcastExchange (59) ++- * ColumnarToRow (58) + +- CometProject (57) + +- CometFilter (56) + +- CometScan parquet spark_catalog.default.date_dim (55) -(49) Scan parquet spark_catalog.default.date_dim +(55) 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 -(50) CometFilter +(56) 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)) -(51) CometProject +(57) 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] -(52) ColumnarToRow [codegen id : 1] +(58) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_moy#11] -(53) BroadcastExchange +(59) 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 = 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) +Subquery:2 Hosting operator id = 27 Hosting Expression = inv_date_sk#30 IN dynamicpruning#31 +BroadcastExchange (64) ++- * ColumnarToRow (63) + +- CometProject (62) + +- CometFilter (61) + +- CometScan parquet spark_catalog.default.date_dim (60) -(54) Scan parquet spark_catalog.default.date_dim +(60) 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 -(55) CometFilter +(61) 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)) -(56) CometProject +(62) 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] -(57) ColumnarToRow [codegen id : 1] +(63) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#35, d_moy#37] -(58) BroadcastExchange +(64) 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 d16db5c8b..48c1cc8b9 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,75 +1,81 @@ WholeStageCodegen (5) - Sort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] + ColumnarToRow InputAdapter - Exchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 - 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 (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] - ColumnarToRow - InputAdapter - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] - CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [i_item_sk] #4 - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [d_date_sk,d_moy] #6 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - 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] + CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] + CometColumnarExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 + RowToColumnar + 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] + ColumnarToRow InputAdapter - 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] - ColumnarToRow - InputAdapter - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] - CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 #9 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - ReusedExchange [i_item_sk] #4 - ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - CometBroadcastExchange [d_date_sk,d_moy] #10 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] + CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [i_item_sk] #4 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange [d_date_sk,d_moy] #6 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] + CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 #9 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + ReusedExchange [i_item_sk] #4 + ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 + CometBroadcastExchange [d_date_sk,d_moy] #10 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + 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 38f56d604..295647e5d 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,52 +1,58 @@ == Physical Plan == -* 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) +* ColumnarToRow (54) ++- CometSort (53) + +- CometColumnarExchange (52) + +- RowToColumnar (51) + +- * BroadcastHashJoin Inner BuildRight (50) + :- * Project (26) + : +- * Filter (25) + : +- * HashAggregate (24) + : +- * ColumnarToRow (23) + : +- CometColumnarExchange (22) + : +- RowToColumnar (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 (49) + +- * Project (48) + +- * Filter (47) + +- * HashAggregate (46) + +- * ColumnarToRow (45) + +- CometColumnarExchange (44) + +- RowToColumnar (43) + +- * HashAggregate (42) + +- * ColumnarToRow (41) + +- CometProject (40) + +- CometBroadcastHashJoin (39) + :- CometProject (34) + : +- CometBroadcastHashJoin (33) + : :- CometProject (31) + : : +- CometBroadcastHashJoin (30) + : : :- CometFilter (28) + : : : +- CometScan parquet spark_catalog.default.inventory (27) + : : +- ReusedExchange (29) + : +- ReusedExchange (32) + +- CometBroadcastExchange (38) + +- CometProject (37) + +- CometFilter (36) + +- CometScan parquet spark_catalog.default.date_dim (35) (1) Scan parquet spark_catalog.default.inventory @@ -147,26 +153,32 @@ Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), par 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] -(21) Exchange +(21) RowToColumnar 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] -(22) HashAggregate [codegen id : 4] +(22) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(23) ColumnarToRow [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] + +(24) 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))#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] -(23) Filter [codegen id : 4] +(25) 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) -(24) Project [codegen id : 4] +(26) 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] -(25) Scan parquet spark_catalog.default.inventory +(27) 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 [] @@ -174,168 +186,180 @@ PartitionFilters: [isnotnull(inv_date_sk#30), dynamicpruningexpression(inv_date_ PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(26) CometFilter +(28) 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)) -(27) ReusedExchange [Reuses operator id: 5] +(29) ReusedExchange [Reuses operator id: 5] Output [1]: [i_item_sk#32] -(28) CometBroadcastHashJoin +(30) 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 -(29) CometProject +(31) 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] -(30) ReusedExchange [Reuses operator id: 10] +(32) ReusedExchange [Reuses operator id: 10] Output [2]: [w_warehouse_sk#33, w_warehouse_name#34] -(31) CometBroadcastHashJoin +(33) 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 -(32) CometProject +(34) 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] -(33) Scan parquet spark_catalog.default.date_dim +(35) 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 -(34) CometFilter +(36) 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 +(37) 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 +(38) CometBroadcastExchange Input [2]: [d_date_sk#35, d_moy#37] Arguments: [d_date_sk#35, d_moy#37] -(37) CometBroadcastHashJoin +(39) 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 +(40) 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] +(41) 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] +(42) 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 +(43) RowToColumnar +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] + +(44) CometColumnarExchange 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] +Arguments: hashpartitioning(w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(42) HashAggregate [codegen id : 3] +(45) ColumnarToRow [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] + +(46) 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] +(47) 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] +(48) 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 +(49) 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] +(50) BroadcastHashJoin [codegen id : 4] Left keys [2]: [i_item_sk#6, w_warehouse_sk#7] Right keys [2]: [i_item_sk#32, w_warehouse_sk#33] Join type: Inner Join condition: None -(47) Exchange +(51) RowToColumnar +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] + +(52) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=4] + +(53) CometSort 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] +Arguments: [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], [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] -(48) Sort [codegen id : 5] +(54) ColumnarToRow [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 (53) -+- * ColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometScan parquet spark_catalog.default.date_dim (49) +BroadcastExchange (59) ++- * ColumnarToRow (58) + +- CometProject (57) + +- CometFilter (56) + +- CometScan parquet spark_catalog.default.date_dim (55) -(49) Scan parquet spark_catalog.default.date_dim +(55) 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 -(50) CometFilter +(56) 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)) -(51) CometProject +(57) 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] -(52) ColumnarToRow [codegen id : 1] +(58) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_moy#11] -(53) BroadcastExchange +(59) 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 = 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) +Subquery:2 Hosting operator id = 27 Hosting Expression = inv_date_sk#30 IN dynamicpruning#31 +BroadcastExchange (64) ++- * ColumnarToRow (63) + +- CometProject (62) + +- CometFilter (61) + +- CometScan parquet spark_catalog.default.date_dim (60) -(54) Scan parquet spark_catalog.default.date_dim +(60) 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 -(55) CometFilter +(61) 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)) -(56) CometProject +(62) 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] -(57) ColumnarToRow [codegen id : 1] +(63) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#35, d_moy#37] -(58) BroadcastExchange +(64) 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 d16db5c8b..48c1cc8b9 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,75 +1,81 @@ WholeStageCodegen (5) - Sort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] + ColumnarToRow InputAdapter - Exchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 - 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 (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] - ColumnarToRow - InputAdapter - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] - CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [i_item_sk] #4 - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [d_date_sk,d_moy] #6 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - 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] + CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] + CometColumnarExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 + RowToColumnar + 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] + ColumnarToRow InputAdapter - 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] - ColumnarToRow - InputAdapter - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] - CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 #9 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - ReusedExchange [i_item_sk] #4 - ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - CometBroadcastExchange [d_date_sk,d_moy] #10 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] + CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [i_item_sk] #4 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange [d_date_sk,d_moy] #6 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] + CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 #9 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + ReusedExchange [i_item_sk] #4 + ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 + CometBroadcastExchange [d_date_sk,d_moy] #10 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + 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 d25660dba..5bbd0d6a4 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,110 +1,122 @@ == Physical Plan == -TakeOrderedAndProject (106) -+- * Project (105) - +- * BroadcastHashJoin Inner BuildRight (104) - :- * Project (88) - : +- * BroadcastHashJoin Inner BuildRight (87) - : :- * Project (70) - : : +- * BroadcastHashJoin Inner BuildRight (69) - : : :- * 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) - : : +- * 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) +TakeOrderedAndProject (118) ++- * Project (117) + +- * BroadcastHashJoin Inner BuildRight (116) + :- * Project (98) + : +- * BroadcastHashJoin Inner BuildRight (97) + : :- * Project (78) + : : +- * BroadcastHashJoin Inner BuildRight (77) + : : :- * Project (59) + : : : +- * BroadcastHashJoin Inner BuildRight (58) + : : : :- * BroadcastHashJoin Inner BuildRight (39) + : : : : :- * Filter (19) + : : : : : +- * HashAggregate (18) + : : : : : +- * ColumnarToRow (17) + : : : : : +- CometColumnarExchange (16) + : : : : : +- RowToColumnar (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 (38) + : : : : +- * HashAggregate (37) + : : : : +- * ColumnarToRow (36) + : : : : +- CometColumnarExchange (35) + : : : : +- RowToColumnar (34) + : : : : +- * HashAggregate (33) + : : : : +- * ColumnarToRow (32) + : : : : +- CometProject (31) + : : : : +- CometBroadcastHashJoin (30) + : : : : :- CometProject (26) + : : : : : +- CometBroadcastHashJoin (25) + : : : : : :- CometFilter (21) + : : : : : : +- CometScan parquet spark_catalog.default.customer (20) + : : : : : +- CometBroadcastExchange (24) + : : : : : +- CometFilter (23) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (22) + : : : : +- CometBroadcastExchange (29) + : : : : +- CometFilter (28) + : : : : +- CometScan parquet spark_catalog.default.date_dim (27) + : : : +- BroadcastExchange (57) + : : : +- * Filter (56) + : : : +- * HashAggregate (55) + : : : +- * ColumnarToRow (54) + : : : +- CometColumnarExchange (53) + : : : +- RowToColumnar (52) + : : : +- * HashAggregate (51) + : : : +- * ColumnarToRow (50) + : : : +- CometProject (49) + : : : +- CometBroadcastHashJoin (48) + : : : :- CometProject (46) + : : : : +- CometBroadcastHashJoin (45) + : : : : :- CometFilter (41) + : : : : : +- CometScan parquet spark_catalog.default.customer (40) + : : : : +- CometBroadcastExchange (44) + : : : : +- CometFilter (43) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (42) + : : : +- ReusedExchange (47) + : : +- BroadcastExchange (76) + : : +- * HashAggregate (75) + : : +- * ColumnarToRow (74) + : : +- CometColumnarExchange (73) + : : +- RowToColumnar (72) + : : +- * HashAggregate (71) + : : +- * ColumnarToRow (70) + : : +- CometProject (69) + : : +- CometBroadcastHashJoin (68) + : : :- CometProject (66) + : : : +- CometBroadcastHashJoin (65) + : : : :- CometFilter (61) + : : : : +- CometScan parquet spark_catalog.default.customer (60) + : : : +- CometBroadcastExchange (64) + : : : +- CometFilter (63) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (62) + : : +- ReusedExchange (67) + : +- BroadcastExchange (96) + : +- * Filter (95) + : +- * HashAggregate (94) + : +- * ColumnarToRow (93) + : +- CometColumnarExchange (92) + : +- RowToColumnar (91) + : +- * HashAggregate (90) + : +- * ColumnarToRow (89) + : +- CometProject (88) + : +- CometBroadcastHashJoin (87) + : :- CometProject (85) + : : +- CometBroadcastHashJoin (84) + : : :- CometFilter (80) + : : : +- CometScan parquet spark_catalog.default.customer (79) + : : +- CometBroadcastExchange (83) + : : +- CometFilter (82) + : : +- CometScan parquet spark_catalog.default.web_sales (81) + : +- ReusedExchange (86) + +- BroadcastExchange (115) + +- * HashAggregate (114) + +- * ColumnarToRow (113) + +- CometColumnarExchange (112) + +- RowToColumnar (111) + +- * HashAggregate (110) + +- * ColumnarToRow (109) + +- CometProject (108) + +- CometBroadcastHashJoin (107) + :- CometProject (105) + : +- CometBroadcastHashJoin (104) + : :- CometFilter (100) + : : +- CometScan parquet spark_catalog.default.customer (99) + : +- CometBroadcastExchange (103) + : +- CometFilter (102) + : +- CometScan parquet spark_catalog.default.web_sales (101) + +- ReusedExchange (106) (1) Scan parquet spark_catalog.default.customer @@ -177,33 +189,39 @@ Functions [1]: [partial_sum(((((ss_ext_list_price#13 - ss_ext_wholesale_cost#12) 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] -(15) Exchange +(15) RowToColumnar 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=1] -(16) HashAggregate [codegen id : 12] +(16) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(17) ColumnarToRow [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] + +(18) 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] -(17) Filter [codegen id : 12] +(19) Filter [codegen id : 12] Input [2]: [customer_id#23, year_total#24] Condition : (isnotnull(year_total#24) AND (year_total#24 > 0.000000)) -(18) Scan parquet spark_catalog.default.customer +(20) 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 -(19) CometFilter +(21) 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)) -(20) Scan parquet spark_catalog.default.store_sales +(22) 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 Location: InMemoryFileIndex [] @@ -211,90 +229,96 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#38), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(21) CometFilter +(23) 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) -(22) CometBroadcastExchange +(24) CometBroadcastExchange 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: [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] -(23) CometBroadcastHashJoin +(25) CometBroadcastHashJoin Left 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] Right 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] Arguments: [c_customer_sk#25], [ss_customer_sk#33], Inner, BuildRight -(24) CometProject +(26) CometProject 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] 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, ss_sold_date_sk#38], [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] -(25) Scan parquet spark_catalog.default.date_dim +(27) 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 -(26) CometFilter +(28) 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 +(29) CometBroadcastExchange Input [2]: [d_date_sk#40, d_year#41] Arguments: [d_date_sk#40, d_year#41] -(28) CometBroadcastHashJoin +(30) 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 +(31) 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] +(32) 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] -(31) HashAggregate [codegen id : 2] +(33) 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] -(32) Exchange +(34) RowToColumnar 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=2] -(33) HashAggregate [codegen id : 3] +(35) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] + +(36) ColumnarToRow [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] + +(37) 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] -(34) BroadcastExchange +(38) 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=3] -(35) BroadcastHashJoin [codegen id : 12] +(39) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#23] Right keys [1]: [customer_id#46] Join type: Inner Join condition: None -(36) Scan parquet spark_catalog.default.customer +(40) 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 -(37) CometFilter +(41) 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)) -(38) Scan parquet spark_catalog.default.catalog_sales +(42) 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 [] @@ -302,86 +326,92 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#67), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(39) CometFilter +(43) 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) -(40) CometBroadcastExchange +(44) 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: [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) CometBroadcastHashJoin +(45) 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) CometProject +(46) 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: 10] +(47) ReusedExchange [Reuses operator id: 10] Output [2]: [d_date_sk#69, d_year#70] -(44) CometBroadcastHashJoin +(48) 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) CometProject +(49) 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) ColumnarToRow [codegen id : 4] +(50) 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] +(51) 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] -(48) Exchange +(52) RowToColumnar +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] + +(53) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=4] + +(54) ColumnarToRow [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] -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] -(49) HashAggregate [codegen id : 5] +(55) 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] -(50) Filter [codegen id : 5] +(56) Filter [codegen id : 5] Input [2]: [customer_id#76, year_total#77] Condition : (isnotnull(year_total#77) AND (year_total#77 > 0.000000)) -(51) BroadcastExchange +(57) BroadcastExchange Input [2]: [customer_id#76, year_total#77] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(52) BroadcastHashJoin [codegen id : 12] +(58) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#23] Right keys [1]: [customer_id#76] Join type: Inner Join condition: None -(53) Project [codegen id : 12] +(59) 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] -(54) Scan parquet spark_catalog.default.customer +(60) 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 -(55) CometFilter +(61) 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)) -(56) Scan parquet spark_catalog.default.catalog_sales +(62) 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 Location: InMemoryFileIndex [] @@ -389,82 +419,88 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#91), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(57) CometFilter +(63) 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) -(58) CometBroadcastExchange +(64) CometBroadcastExchange 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: [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] -(59) CometBroadcastHashJoin +(65) CometBroadcastHashJoin Left 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] Right 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] Arguments: [c_customer_sk#78], [cs_bill_customer_sk#86], Inner, BuildRight -(60) CometProject +(66) CometProject 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] Arguments: [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], [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] -(61) ReusedExchange [Reuses operator id: 27] +(67) ReusedExchange [Reuses operator id: 29] Output [2]: [d_date_sk#93, d_year#94] -(62) CometBroadcastHashJoin +(68) CometBroadcastHashJoin Left 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] Right output [2]: [d_date_sk#93, d_year#94] Arguments: [cs_sold_date_sk#91], [d_date_sk#93], Inner, BuildRight -(63) CometProject +(69) CometProject 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] Arguments: [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], [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] -(64) ColumnarToRow [codegen id : 6] +(70) ColumnarToRow [codegen id : 6] 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] -(65) HashAggregate [codegen id : 6] +(71) HashAggregate [codegen id : 6] 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] -(66) Exchange +(72) RowToColumnar +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] + +(73) CometColumnarExchange 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=6] +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, CometColumnarShuffle, [plan_id=6] -(67) HashAggregate [codegen id : 7] +(74) ColumnarToRow [codegen id : 7] +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] + +(75) HashAggregate [codegen id : 7] 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] -(68) BroadcastExchange +(76) BroadcastExchange Input [2]: [customer_id#99, year_total#100] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] -(69) BroadcastHashJoin [codegen id : 12] +(77) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#23] Right keys [1]: [customer_id#99] 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) -(70) Project [codegen id : 12] +(78) 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] -(71) Scan parquet spark_catalog.default.customer +(79) 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] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(72) CometFilter +(80) 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) Scan parquet spark_catalog.default.web_sales +(81) 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] Batched: true Location: InMemoryFileIndex [] @@ -472,86 +508,92 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#114), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(74) CometFilter +(82) 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) -(75) CometBroadcastExchange +(83) 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] -(76) CometBroadcastHashJoin +(84) 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 -(77) CometProject +(85) 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] -(78) ReusedExchange [Reuses operator id: 10] +(86) ReusedExchange [Reuses operator id: 10] Output [2]: [d_date_sk#116, d_year#117] -(79) CometBroadcastHashJoin +(87) 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 -(80) CometProject +(88) 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] +(89) 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] -(82) HashAggregate [codegen id : 8] +(90) 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] -(83) Exchange +(91) RowToColumnar +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] + +(92) CometColumnarExchange 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=8] +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, CometColumnarShuffle, [plan_id=8] -(84) HashAggregate [codegen id : 9] +(93) ColumnarToRow [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] + +(94) 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] -(85) Filter [codegen id : 9] +(95) Filter [codegen id : 9] Input [2]: [customer_id#123, year_total#124] Condition : (isnotnull(year_total#124) AND (year_total#124 > 0.000000)) -(86) BroadcastExchange +(96) BroadcastExchange Input [2]: [customer_id#123, year_total#124] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=9] -(87) BroadcastHashJoin [codegen id : 12] +(97) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#23] Right keys [1]: [customer_id#123] Join type: Inner Join condition: None -(88) Project [codegen id : 12] +(98) 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] -(89) Scan parquet spark_catalog.default.customer +(99) 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 -(90) CometFilter +(100) 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)) -(91) Scan parquet spark_catalog.default.web_sales +(101) 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 [] @@ -559,132 +601,138 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#138), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(92) CometFilter +(102) 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) -(93) CometBroadcastExchange +(103) 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] -(94) CometBroadcastHashJoin +(104) 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 -(95) CometProject +(105) 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] -(96) ReusedExchange [Reuses operator id: 27] +(106) ReusedExchange [Reuses operator id: 29] Output [2]: [d_date_sk#140, d_year#141] -(97) CometBroadcastHashJoin +(107) 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 -(98) CometProject +(108) 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] +(109) 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] -(100) HashAggregate [codegen id : 10] +(110) 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] -(101) Exchange +(111) RowToColumnar +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] + +(112) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=10] + +(113) ColumnarToRow [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] -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] -(102) HashAggregate [codegen id : 11] +(114) 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] -(103) BroadcastExchange +(115) BroadcastExchange Input [2]: [customer_id#146, year_total#147] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(104) BroadcastHashJoin [codegen id : 12] +(116) 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) -(105) Project [codegen id : 12] +(117) 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] -(106) TakeOrderedAndProject +(118) 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 = 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) +BroadcastExchange (122) ++- * ColumnarToRow (121) + +- CometFilter (120) + +- CometScan parquet spark_catalog.default.date_dim (119) -(107) Scan parquet spark_catalog.default.date_dim +(119) 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 -(108) CometFilter +(120) 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)) -(109) ColumnarToRow [codegen id : 1] +(121) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#16, d_year#17] -(110) BroadcastExchange +(122) BroadcastExchange Input [2]: [d_date_sk#16, d_year#17] 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 (114) -+- * ColumnarToRow (113) - +- CometFilter (112) - +- CometScan parquet spark_catalog.default.date_dim (111) +Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#38 IN dynamicpruning#39 +BroadcastExchange (126) ++- * ColumnarToRow (125) + +- CometFilter (124) + +- CometScan parquet spark_catalog.default.date_dim (123) -(111) Scan parquet spark_catalog.default.date_dim +(123) 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 -(112) CometFilter +(124) 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)) -(113) ColumnarToRow [codegen id : 1] +(125) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#40, d_year#41] -(114) BroadcastExchange +(126) BroadcastExchange Input [2]: [d_date_sk#40, d_year#41] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -Subquery:3 Hosting operator id = 38 Hosting Expression = cs_sold_date_sk#67 IN dynamicpruning#15 +Subquery:3 Hosting operator id = 42 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:4 Hosting operator id = 62 Hosting Expression = cs_sold_date_sk#91 IN dynamicpruning#39 -Subquery:5 Hosting operator id = 73 Hosting Expression = ws_sold_date_sk#114 IN dynamicpruning#15 +Subquery:5 Hosting operator id = 81 Hosting Expression = ws_sold_date_sk#114 IN dynamicpruning#15 -Subquery:6 Hosting operator id = 91 Hosting Expression = ws_sold_date_sk#138 IN dynamicpruning#39 +Subquery:6 Hosting operator id = 101 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 a412a54d9..9f8d70dcb 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 @@ -11,38 +11,11 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom BroadcastHashJoin [customer_id,customer_id] 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(((((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 (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] - 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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_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_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - 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 + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #1 + RowToColumnar + 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] ColumnarToRow InputAdapter @@ -52,102 +25,141 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] 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 [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #7 + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #2 CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_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 #8 + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 + CometBroadcastExchange [d_date_sk,d_year] #4 CometFilter [d_date_sk,d_year] 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #6 + RowToColumnar + 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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + 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 [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_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 #8 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter 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] #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] - 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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #12 - CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_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 + ColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 + RowToColumnar + 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] + 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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + 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 [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #12 + CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_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 #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] #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] - 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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #15 - CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_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 + ColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 + RowToColumnar + 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] + 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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + 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 [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #15 + CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_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 #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] #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] - 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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #18 - CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_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 + ColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #17 + RowToColumnar + 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] + 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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #18 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_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 #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] #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] - 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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #21 - CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_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 + ColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #20 + RowToColumnar + 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] + 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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #21 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_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/q40/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/explain.txt index f63b94658..b962258c5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/explain.txt @@ -1,37 +1,38 @@ == Physical Plan == -TakeOrderedAndProject (33) -+- * HashAggregate (32) - +- Exchange (31) - +- * HashAggregate (30) - +- * Project (29) - +- * BroadcastHashJoin Inner BuildRight (28) - :- * Project (26) - : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (13) - : : : +- * SortMergeJoin LeftOuter (12) - : : : :- * Sort (5) - : : : : +- Exchange (4) - : : : : +- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : +- * Sort (11) - : : : +- Exchange (10) - : : : +- * ColumnarToRow (9) - : : : +- CometProject (8) - : : : +- CometFilter (7) - : : : +- CometScan parquet spark_catalog.default.catalog_returns (6) - : : +- BroadcastExchange (17) - : : +- * ColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometScan parquet spark_catalog.default.warehouse (14) - : +- BroadcastExchange (24) - : +- * ColumnarToRow (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan parquet spark_catalog.default.item (20) - +- ReusedExchange (27) +TakeOrderedAndProject (34) ++- * HashAggregate (33) + +- * ColumnarToRow (32) + +- CometColumnarExchange (31) + +- RowToColumnar (30) + +- * HashAggregate (29) + +- * ColumnarToRow (28) + +- CometProject (27) + +- CometBroadcastHashJoin (26) + :- CometProject (22) + : +- CometBroadcastHashJoin (21) + : :- CometProject (16) + : : +- CometBroadcastHashJoin (15) + : : :- CometProject (11) + : : : +- CometSortMergeJoin (10) + : : : :- CometSort (4) + : : : : +- CometColumnarExchange (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : +- CometSort (9) + : : : +- CometColumnarExchange (8) + : : : +- CometProject (7) + : : : +- CometFilter (6) + : : : +- CometScan parquet spark_catalog.default.catalog_returns (5) + : : +- CometBroadcastExchange (14) + : : +- CometFilter (13) + : : +- CometScan parquet spark_catalog.default.warehouse (12) + : +- CometBroadcastExchange (20) + : +- CometProject (19) + : +- CometFilter (18) + : +- CometScan parquet spark_catalog.default.item (17) + +- CometBroadcastExchange (25) + +- CometFilter (24) + +- CometScan parquet spark_catalog.default.date_dim (23) (1) Scan parquet spark_catalog.default.catalog_sales @@ -46,173 +47,178 @@ ReadSchema: struct -(7) CometFilter +(6) CometFilter Input [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] Condition : (isnotnull(cr_order_number#8) AND isnotnull(cr_item_sk#7)) -(8) CometProject +(7) CometProject Input [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] Arguments: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9], [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -(9) ColumnarToRow [codegen id : 3] -Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] - -(10) Exchange +(8) CometColumnarExchange Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: hashpartitioning(cr_order_number#8, cr_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(cr_order_number#8, cr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(11) Sort [codegen id : 4] +(9) CometSort Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cr_order_number#8 ASC NULLS FIRST, cr_item_sk#7 ASC NULLS FIRST], false, 0 +Arguments: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9], [cr_order_number#8 ASC NULLS FIRST, cr_item_sk#7 ASC NULLS FIRST] -(12) SortMergeJoin [codegen id : 8] -Left keys [2]: [cs_order_number#3, cs_item_sk#2] -Right keys [2]: [cr_order_number#8, cr_item_sk#7] -Join type: LeftOuter -Join condition: None +(10) CometSortMergeJoin +Left output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] +Right output [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] +Arguments: [cs_order_number#3, cs_item_sk#2], [cr_order_number#8, cr_item_sk#7], LeftOuter -(13) Project [codegen id : 8] -Output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9] +(11) CometProject Input [8]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5, cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] +Arguments: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9], [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9] -(14) Scan parquet spark_catalog.default.warehouse +(12) Scan parquet spark_catalog.default.warehouse Output [2]: [w_warehouse_sk#11, w_state#12] Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] PushedFilters: [IsNotNull(w_warehouse_sk)] ReadSchema: struct -(15) CometFilter +(13) CometFilter Input [2]: [w_warehouse_sk#11, w_state#12] Condition : isnotnull(w_warehouse_sk#11) -(16) ColumnarToRow [codegen id : 5] +(14) CometBroadcastExchange Input [2]: [w_warehouse_sk#11, w_state#12] +Arguments: [w_warehouse_sk#11, w_state#12] -(17) BroadcastExchange -Input [2]: [w_warehouse_sk#11, w_state#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(18) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_warehouse_sk#1] -Right keys [1]: [w_warehouse_sk#11] -Join type: Inner -Join condition: None +(15) CometBroadcastHashJoin +Left output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9] +Right output [2]: [w_warehouse_sk#11, w_state#12] +Arguments: [cs_warehouse_sk#1], [w_warehouse_sk#11], Inner, BuildRight -(19) Project [codegen id : 8] -Output [5]: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12] +(16) CometProject Input [7]: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_warehouse_sk#11, w_state#12] +Arguments: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12], [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12] -(20) Scan parquet spark_catalog.default.item +(17) Scan parquet spark_catalog.default.item Output [3]: [i_item_sk#13, i_item_id#14, i_current_price#15] 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 -(21) CometFilter +(18) CometFilter Input [3]: [i_item_sk#13, i_item_id#14, i_current_price#15] Condition : (((isnotnull(i_current_price#15) AND (i_current_price#15 >= 0.99)) AND (i_current_price#15 <= 1.49)) AND isnotnull(i_item_sk#13)) -(22) CometProject +(19) CometProject Input [3]: [i_item_sk#13, i_item_id#14, i_current_price#15] Arguments: [i_item_sk#13, i_item_id#14], [i_item_sk#13, i_item_id#14] -(23) ColumnarToRow [codegen id : 6] +(20) CometBroadcastExchange Input [2]: [i_item_sk#13, i_item_id#14] +Arguments: [i_item_sk#13, i_item_id#14] -(24) BroadcastExchange -Input [2]: [i_item_sk#13, i_item_id#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(25) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_item_sk#2] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None +(21) CometBroadcastHashJoin +Left output [5]: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12] +Right output [2]: [i_item_sk#13, i_item_id#14] +Arguments: [cs_item_sk#2], [i_item_sk#13], Inner, BuildRight -(26) Project [codegen id : 8] -Output [5]: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12, i_item_id#14] +(22) CometProject Input [7]: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12, i_item_sk#13, i_item_id#14] +Arguments: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12, i_item_id#14], [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12, i_item_id#14] -(27) ReusedExchange [Reuses operator id: 37] +(23) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#16, d_date#17] +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 + +(24) CometFilter +Input [2]: [d_date_sk#16, d_date#17] +Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 2000-02-10)) AND (d_date#17 <= 2000-04-10)) AND isnotnull(d_date_sk#16)) + +(25) CometBroadcastExchange +Input [2]: [d_date_sk#16, d_date#17] +Arguments: [d_date_sk#16, d_date#17] -(28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#16] -Join type: Inner -Join condition: None +(26) CometBroadcastHashJoin +Left output [5]: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12, i_item_id#14] +Right output [2]: [d_date_sk#16, d_date#17] +Arguments: [cs_sold_date_sk#5], [d_date_sk#16], Inner, BuildRight -(29) Project [codegen id : 8] -Output [5]: [cs_sales_price#4, cr_refunded_cash#9, w_state#12, i_item_id#14, d_date#17] +(27) CometProject Input [7]: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12, i_item_id#14, d_date_sk#16, d_date#17] +Arguments: [cs_sales_price#4, cr_refunded_cash#9, w_state#12, i_item_id#14, d_date#17], [cs_sales_price#4, cr_refunded_cash#9, w_state#12, i_item_id#14, d_date#17] -(30) HashAggregate [codegen id : 8] +(28) ColumnarToRow [codegen id : 1] +Input [5]: [cs_sales_price#4, cr_refunded_cash#9, w_state#12, i_item_id#14, d_date#17] + +(29) HashAggregate [codegen id : 1] Input [5]: [cs_sales_price#4, cr_refunded_cash#9, w_state#12, i_item_id#14, d_date#17] Keys [2]: [w_state#12, i_item_id#14] Functions [2]: [partial_sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END), partial_sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)] Aggregate Attributes [4]: [sum#18, isEmpty#19, sum#20, isEmpty#21] Results [6]: [w_state#12, i_item_id#14, sum#22, isEmpty#23, sum#24, isEmpty#25] -(31) Exchange +(30) RowToColumnar +Input [6]: [w_state#12, i_item_id#14, sum#22, isEmpty#23, sum#24, isEmpty#25] + +(31) CometColumnarExchange +Input [6]: [w_state#12, i_item_id#14, sum#22, isEmpty#23, sum#24, isEmpty#25] +Arguments: hashpartitioning(w_state#12, i_item_id#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(32) ColumnarToRow [codegen id : 2] Input [6]: [w_state#12, i_item_id#14, sum#22, isEmpty#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(w_state#12, i_item_id#14, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(32) HashAggregate [codegen id : 9] +(33) HashAggregate [codegen id : 2] Input [6]: [w_state#12, i_item_id#14, sum#22, isEmpty#23, sum#24, isEmpty#25] Keys [2]: [w_state#12, i_item_id#14] Functions [2]: [sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END), sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)] Aggregate Attributes [2]: [sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#26, sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#27] Results [4]: [w_state#12, i_item_id#14, sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#26 AS sales_before#28, sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#27 AS sales_after#29] -(33) TakeOrderedAndProject +(34) TakeOrderedAndProject Input [4]: [w_state#12, i_item_id#14, sales_before#28, sales_after#29] Arguments: 100, [w_state#12 ASC NULLS FIRST, i_item_id#14 ASC NULLS FIRST], [w_state#12, i_item_id#14, sales_before#28, sales_after#29] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (37) -+- * ColumnarToRow (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.date_dim (34) +BroadcastExchange (38) ++- * ColumnarToRow (37) + +- CometFilter (36) + +- CometScan parquet spark_catalog.default.date_dim (35) -(34) Scan parquet spark_catalog.default.date_dim +(35) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#16, d_date#17] 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 -(35) CometFilter +(36) CometFilter Input [2]: [d_date_sk#16, d_date#17] Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 2000-02-10)) AND (d_date#17 <= 2000-04-10)) AND isnotnull(d_date_sk#16)) -(36) ColumnarToRow [codegen id : 1] +(37) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#16, d_date#17] -(37) BroadcastExchange +(38) BroadcastExchange Input [2]: [d_date_sk#16, d_date#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/simplified.txt index 11c0201a8..d5ce3bfe8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/simplified.txt @@ -1,60 +1,45 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] - WholeStageCodegen (9) + WholeStageCodegen (2) HashAggregate [w_state,i_item_id,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_date < 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END),sales_before,sales_after,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [w_state,i_item_id] #1 - WholeStageCodegen (8) - HashAggregate [w_state,i_item_id,d_date,cs_sales_price,cr_refunded_cash] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Project [cs_sales_price,cr_refunded_cash,w_state,i_item_id,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state] - BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - Project [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash] - SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - InputAdapter - WholeStageCodegen (2) - Sort [cs_order_number,cs_item_sk] - InputAdapter - Exchange [cs_order_number,cs_item_sk] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - WholeStageCodegen (4) - Sort [cr_order_number,cr_item_sk] - InputAdapter - Exchange [cr_order_number,cr_item_sk] #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometColumnarExchange [w_state,i_item_id] #1 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [w_state,i_item_id,d_date,cs_sales_price,cr_refunded_cash] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometProject [cs_sales_price,cr_refunded_cash,w_state,i_item_id,d_date] + CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_id,d_date_sk,d_date] + CometProject [cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_id] + CometBroadcastHashJoin [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_sk,i_item_id] + CometProject [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state] + CometBroadcastHashJoin [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_warehouse_sk,w_state] + CometProject [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash] + CometSortMergeJoin [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_refunded_cash] + CometSort [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] + CometColumnarExchange [cs_order_number,cs_item_sk] #2 + CometFilter [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [cr_item_sk,cr_order_number,cr_refunded_cash] + CometColumnarExchange [cr_order_number,cr_item_sk] #4 + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] + CometBroadcastExchange [w_warehouse_sk,w_state] #5 CometFilter [w_warehouse_sk,w_state] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter + CometBroadcastExchange [i_item_sk,i_item_id] #6 CometProject [i_item_sk,i_item_id] CometFilter [i_item_sk,i_item_id,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 + CometBroadcastExchange [d_date_sk,d_date] #7 + CometFilter [d_date_sk,d_date] + 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/q41/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/explain.txt index 07196ba8c..5c2b28b9d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/explain.txt @@ -1,24 +1,23 @@ == Physical Plan == -TakeOrderedAndProject (20) -+- * HashAggregate (19) - +- Exchange (18) - +- * HashAggregate (17) - +- * Project (16) - +- * BroadcastHashJoin Inner BuildRight (15) - :- * ColumnarToRow (4) - : +- CometProject (3) - : +- CometFilter (2) - : +- CometScan parquet spark_catalog.default.item (1) - +- BroadcastExchange (14) - +- * Project (13) - +- * Filter (12) - +- * HashAggregate (11) - +- Exchange (10) - +- * ColumnarToRow (9) - +- CometHashAggregate (8) - +- CometProject (7) - +- CometFilter (6) - +- CometScan parquet spark_catalog.default.item (5) +* ColumnarToRow (19) ++- CometTakeOrderedAndProject (18) + +- CometHashAggregate (17) + +- CometColumnarExchange (16) + +- CometHashAggregate (15) + +- CometProject (14) + +- CometBroadcastHashJoin (13) + :- CometProject (3) + : +- CometFilter (2) + : +- CometScan parquet spark_catalog.default.item (1) + +- CometBroadcastExchange (12) + +- CometProject (11) + +- CometFilter (10) + +- CometHashAggregate (9) + +- CometColumnarExchange (8) + +- CometHashAggregate (7) + +- CometProject (6) + +- CometFilter (5) + +- CometScan parquet spark_catalog.default.item (4) (1) Scan parquet spark_catalog.default.item @@ -36,84 +35,74 @@ Condition : (((isnotnull(i_manufact_id#1) AND (i_manufact_id#1 >= 738)) AND (i_m Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] Arguments: [i_manufact#2, i_product_name#3], [i_manufact#2, i_product_name#3] -(4) ColumnarToRow [codegen id : 3] -Input [2]: [i_manufact#2, i_product_name#3] - -(5) Scan parquet spark_catalog.default.item +(4) Scan parquet spark_catalog.default.item Output [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [Or(Or(And(EqualTo(i_category,Women ),Or(And(And(Or(EqualTo(i_color,powder ),EqualTo(i_color,khaki )),Or(EqualTo(i_units,Ounce ),EqualTo(i_units,Oz ))),Or(EqualTo(i_size,medium ),EqualTo(i_size,extra large ))),And(And(Or(EqualTo(i_color,brown ),EqualTo(i_color,honeydew )),Or(EqualTo(i_units,Bunch ),EqualTo(i_units,Ton ))),Or(EqualTo(i_size,N/A ),EqualTo(i_size,small ))))),And(EqualTo(i_category,Men ),Or(And(And(Or(EqualTo(i_color,floral ),EqualTo(i_color,deep )),Or(EqualTo(i_units,N/A ),EqualTo(i_units,Dozen ))),Or(EqualTo(i_size,petite ),EqualTo(i_size,large ))),And(And(Or(EqualTo(i_color,light ),EqualTo(i_color,cornflower )),Or(EqualTo(i_units,Box ),EqualTo(i_units,Pound ))),Or(EqualTo(i_size,medium ),EqualTo(i_size,extra large )))))),Or(And(EqualTo(i_category,Women ),Or(And(And(Or(EqualTo(i_color,midnight ),EqualTo(i_color,snow )),Or(EqualTo(i_units,Pallet ),EqualTo(i_units,Gross ))),Or(EqualTo(i_size,medium ),EqualTo(i_size,extra large ))),And(And(Or(EqualTo(i_color,cyan ),EqualTo(i_color,papaya )),Or(EqualTo(i_units,Cup ),EqualTo(i_units,Dram ))),Or(EqualTo(i_size,N/A ),EqualTo(i_size,small ))))),And(EqualTo(i_category,Men ),Or(And(And(Or(EqualTo(i_color,orange ),EqualTo(i_color,frosted )),Or(EqualTo(i_units,Each ),EqualTo(i_units,Tbl ))),Or(EqualTo(i_size,petite ),EqualTo(i_size,large ))),And(And(Or(EqualTo(i_color,forest ),EqualTo(i_color,ghost )),Or(EqualTo(i_units,Lb ),EqualTo(i_units,Bundle ))),Or(EqualTo(i_size,medium ),EqualTo(i_size,extra large ))))))), IsNotNull(i_manufact)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] Condition : (((((i_category#4 = Women ) AND (((((i_color#7 = powder ) OR (i_color#7 = khaki )) AND ((i_units#8 = Ounce ) OR (i_units#8 = Oz ))) AND ((i_size#6 = medium ) OR (i_size#6 = extra large ))) OR ((((i_color#7 = brown ) OR (i_color#7 = honeydew )) AND ((i_units#8 = Bunch ) OR (i_units#8 = Ton ))) AND ((i_size#6 = N/A ) OR (i_size#6 = small ))))) OR ((i_category#4 = Men ) AND (((((i_color#7 = floral ) OR (i_color#7 = deep )) AND ((i_units#8 = N/A ) OR (i_units#8 = Dozen ))) AND ((i_size#6 = petite ) OR (i_size#6 = large ))) OR ((((i_color#7 = light ) OR (i_color#7 = cornflower )) AND ((i_units#8 = Box ) OR (i_units#8 = Pound ))) AND ((i_size#6 = medium ) OR (i_size#6 = extra large )))))) OR (((i_category#4 = Women ) AND (((((i_color#7 = midnight ) OR (i_color#7 = snow )) AND ((i_units#8 = Pallet ) OR (i_units#8 = Gross ))) AND ((i_size#6 = medium ) OR (i_size#6 = extra large ))) OR ((((i_color#7 = cyan ) OR (i_color#7 = papaya )) AND ((i_units#8 = Cup ) OR (i_units#8 = Dram ))) AND ((i_size#6 = N/A ) OR (i_size#6 = small ))))) OR ((i_category#4 = Men ) AND (((((i_color#7 = orange ) OR (i_color#7 = frosted )) AND ((i_units#8 = Each ) OR (i_units#8 = Tbl ))) AND ((i_size#6 = petite ) OR (i_size#6 = large ))) OR ((((i_color#7 = forest ) OR (i_color#7 = ghost )) AND ((i_units#8 = Lb ) OR (i_units#8 = Bundle ))) AND ((i_size#6 = medium ) OR (i_size#6 = extra large ))))))) AND isnotnull(i_manufact#5)) -(7) CometProject +(6) CometProject Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] Arguments: [i_manufact#5], [i_manufact#5] -(8) CometHashAggregate +(7) CometHashAggregate Input [1]: [i_manufact#5] Keys [1]: [i_manufact#5] Functions [1]: [partial_count(1)] -(9) ColumnarToRow [codegen id : 1] -Input [2]: [i_manufact#5, count#9] - -(10) Exchange +(8) CometColumnarExchange Input [2]: [i_manufact#5, count#9] -Arguments: hashpartitioning(i_manufact#5, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: hashpartitioning(i_manufact#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(11) HashAggregate [codegen id : 2] +(9) CometHashAggregate Input [2]: [i_manufact#5, count#9] Keys [1]: [i_manufact#5] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#10] -Results [2]: [count(1)#10 AS item_cnt#11, i_manufact#5] -(12) Filter [codegen id : 2] -Input [2]: [item_cnt#11, i_manufact#5] -Condition : (item_cnt#11 > 0) +(10) CometFilter +Input [2]: [item_cnt#10, i_manufact#5] +Condition : (item_cnt#10 > 0) -(13) Project [codegen id : 2] -Output [1]: [i_manufact#5] -Input [2]: [item_cnt#11, i_manufact#5] +(11) CometProject +Input [2]: [item_cnt#10, i_manufact#5] +Arguments: [i_manufact#5], [i_manufact#5] -(14) BroadcastExchange +(12) CometBroadcastExchange Input [1]: [i_manufact#5] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] +Arguments: [i_manufact#5] -(15) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [i_manufact#2] -Right keys [1]: [i_manufact#5] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [2]: [i_manufact#2, i_product_name#3] +Right output [1]: [i_manufact#5] +Arguments: [i_manufact#2], [i_manufact#5], Inner, BuildRight -(16) Project [codegen id : 3] -Output [1]: [i_product_name#3] +(14) CometProject Input [3]: [i_manufact#2, i_product_name#3, i_manufact#5] +Arguments: [i_product_name#3], [i_product_name#3] -(17) HashAggregate [codegen id : 3] +(15) CometHashAggregate Input [1]: [i_product_name#3] Keys [1]: [i_product_name#3] Functions: [] -Aggregate Attributes: [] -Results [1]: [i_product_name#3] -(18) Exchange +(16) CometColumnarExchange Input [1]: [i_product_name#3] -Arguments: hashpartitioning(i_product_name#3, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(i_product_name#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(19) HashAggregate [codegen id : 4] +(17) CometHashAggregate Input [1]: [i_product_name#3] Keys [1]: [i_product_name#3] Functions: [] -Aggregate Attributes: [] -Results [1]: [i_product_name#3] -(20) TakeOrderedAndProject +(18) CometTakeOrderedAndProject +Input [1]: [i_product_name#3] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_product_name#3 ASC NULLS FIRST], output=[i_product_name#3]), [i_product_name#3], 100, [i_product_name#3 ASC NULLS FIRST], [i_product_name#3] + +(19) ColumnarToRow [codegen id : 1] Input [1]: [i_product_name#3] -Arguments: 100, [i_product_name#3 ASC NULLS FIRST], [i_product_name#3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/simplified.txt index 457d441ce..732e8679b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/simplified.txt @@ -1,29 +1,21 @@ -TakeOrderedAndProject [i_product_name] - WholeStageCodegen (4) - HashAggregate [i_product_name] - InputAdapter - Exchange [i_product_name] #1 - WholeStageCodegen (3) - HashAggregate [i_product_name] - Project [i_product_name] - BroadcastHashJoin [i_manufact,i_manufact] - ColumnarToRow - InputAdapter - CometProject [i_manufact,i_product_name] - CometFilter [i_manufact_id,i_manufact,i_product_name] - CometScan parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [i_manufact] - Filter [item_cnt] - HashAggregate [i_manufact,count] [count(1),item_cnt,count] - InputAdapter - Exchange [i_manufact] #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_manufact,count] - CometProject [i_manufact] - CometFilter [i_category,i_manufact,i_size,i_color,i_units] - CometScan parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_product_name] + CometHashAggregate [i_product_name] + CometColumnarExchange [i_product_name] #1 + CometHashAggregate [i_product_name] + CometProject [i_product_name] + CometBroadcastHashJoin [i_manufact,i_product_name,i_manufact] + CometProject [i_manufact,i_product_name] + CometFilter [i_manufact_id,i_manufact,i_product_name] + CometScan parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] + CometBroadcastExchange [i_manufact] #2 + CometProject [i_manufact] + CometFilter [item_cnt,i_manufact] + CometHashAggregate [item_cnt,i_manufact,count,count(1)] + CometColumnarExchange [i_manufact] #3 + CometHashAggregate [i_manufact,count] + CometProject [i_manufact] + CometFilter [i_category,i_manufact,i_size,i_color,i_units] + CometScan parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] 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 a43b8716f..121d38dcd 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,23 +1,25 @@ == Physical Plan == -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) +TakeOrderedAndProject (21) ++- * HashAggregate (20) + +- * ColumnarToRow (19) + +- CometColumnarExchange (18) + +- RowToColumnar (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 @@ -98,18 +100,24 @@ 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] -(17) Exchange +(17) RowToColumnar 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=1] -(18) HashAggregate [codegen id : 2] +(18) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(19) ColumnarToRow [codegen id : 2] +Input [4]: [d_year#2, i_category_id#8, i_category#9, sum#12] + +(20) 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] -(19) TakeOrderedAndProject +(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] 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 d6e3733ce..edc1cc2f5 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,23 +1,25 @@ TakeOrderedAndProject [sum(ss_ext_sales_price),d_year,i_category_id,i_category] 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 (1) - HashAggregate [d_year,i_category_id,i_category,ss_ext_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [d_year,ss_ext_sales_price,i_category_id,i_category] - CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_category_id,i_category] - CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_category_id,i_category] #3 - CometProject [i_item_sk,i_category_id,i_category] - CometFilter [i_item_sk,i_category_id,i_category,i_manager_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_category_id,i_category,i_manager_id] + ColumnarToRow + InputAdapter + CometColumnarExchange [d_year,i_category_id,i_category] #1 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [d_year,i_category_id,i_category,ss_ext_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [d_year,ss_ext_sales_price,i_category_id,i_category] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_category_id,i_category] + CometProject [d_year,ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometProject [d_date_sk,d_year] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange [i_item_sk,i_category_id,i_category] #3 + CometProject [i_item_sk,i_category_id,i_category] + CometFilter [i_item_sk,i_category_id,i_category,i_manager_id] + 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 abe134d26..a72d6cc7d 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,23 +1,25 @@ == Physical Plan == -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) +TakeOrderedAndProject (21) ++- * HashAggregate (20) + +- * ColumnarToRow (19) + +- CometColumnarExchange (18) + +- RowToColumnar (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 @@ -98,18 +100,24 @@ Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday ) T 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] -(17) Exchange +(17) RowToColumnar 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=1] -(18) HashAggregate [codegen id : 2] +(18) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(19) ColumnarToRow [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] + +(20) 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] -(19) TakeOrderedAndProject +(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] 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 f21d846c0..72d437599 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,23 +1,25 @@ TakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] 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 (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] - ColumnarToRow - InputAdapter - CometProject [d_day_name,ss_sales_price,s_store_id,s_store_name] - CometBroadcastHashJoin [d_day_name,ss_store_sk,ss_sales_price,s_store_sk,s_store_id,s_store_name] - CometProject [d_day_name,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [d_date_sk,d_day_name,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_day_name] - CometFilter [d_date_sk,d_year,d_day_name] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] - CometBroadcastExchange [ss_store_sk,ss_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 - CometProject [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name,s_gmt_offset] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset] + ColumnarToRow + InputAdapter + CometColumnarExchange [s_store_name,s_store_id] #1 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometProject [d_day_name,ss_sales_price,s_store_id,s_store_name] + CometBroadcastHashJoin [d_day_name,ss_store_sk,ss_sales_price,s_store_sk,s_store_id,s_store_name] + CometProject [d_day_name,ss_store_sk,ss_sales_price] + CometBroadcastHashJoin [d_date_sk,d_day_name,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [d_date_sk,d_day_name] + CometFilter [d_date_sk,d_year,d_day_name] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] + CometBroadcastExchange [ss_store_sk,ss_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 + CometProject [s_store_sk,s_store_id,s_store_name] + CometFilter [s_store_sk,s_store_id,s_store_name,s_gmt_offset] + 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/q44/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/explain.txt index 812f9f391..de02788f4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/explain.txt @@ -1,47 +1,56 @@ == Physical Plan == -TakeOrderedAndProject (43) -+- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (33) - : : +- * SortMergeJoin Inner (32) - : : :- * Sort (18) - : : : +- Exchange (17) - : : : +- * Project (16) - : : : +- * Filter (15) - : : : +- Window (14) - : : : +- WindowGroupLimit (13) - : : : +- * Sort (12) - : : : +- Exchange (11) - : : : +- WindowGroupLimit (10) - : : : +- * Sort (9) - : : : +- * Filter (8) - : : : +- * HashAggregate (7) - : : : +- Exchange (6) - : : : +- * HashAggregate (5) - : : : +- * ColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : +- * Sort (31) - : : +- Exchange (30) - : : +- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- WindowGroupLimit (26) - : : +- * Sort (25) - : : +- Exchange (24) - : : +- WindowGroupLimit (23) - : : +- * Sort (22) - : : +- * Filter (21) - : : +- * HashAggregate (20) - : : +- ReusedExchange (19) - : +- BroadcastExchange (37) - : +- * ColumnarToRow (36) - : +- CometFilter (35) - : +- CometScan parquet spark_catalog.default.item (34) - +- ReusedExchange (40) +* ColumnarToRow (52) ++- CometTakeOrderedAndProject (51) + +- CometProject (50) + +- CometBroadcastHashJoin (49) + :- CometProject (47) + : +- CometBroadcastHashJoin (46) + : :- CometProject (42) + : : +- CometSortMergeJoin (41) + : : :- CometSort (23) + : : : +- CometColumnarExchange (22) + : : : +- RowToColumnar (21) + : : : +- * Project (20) + : : : +- * Filter (19) + : : : +- Window (18) + : : : +- WindowGroupLimit (17) + : : : +- * ColumnarToRow (16) + : : : +- CometSort (15) + : : : +- CometColumnarExchange (14) + : : : +- RowToColumnar (13) + : : : +- WindowGroupLimit (12) + : : : +- * Sort (11) + : : : +- * Filter (10) + : : : +- * HashAggregate (9) + : : : +- * ColumnarToRow (8) + : : : +- CometColumnarExchange (7) + : : : +- RowToColumnar (6) + : : : +- * HashAggregate (5) + : : : +- * ColumnarToRow (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- CometSort (40) + : : +- CometColumnarExchange (39) + : : +- RowToColumnar (38) + : : +- * Project (37) + : : +- * Filter (36) + : : +- Window (35) + : : +- WindowGroupLimit (34) + : : +- * ColumnarToRow (33) + : : +- CometSort (32) + : : +- CometColumnarExchange (31) + : : +- RowToColumnar (30) + : : +- WindowGroupLimit (29) + : : +- * Sort (28) + : : +- * Filter (27) + : : +- * HashAggregate (26) + : : +- * ColumnarToRow (25) + : : +- ReusedExchange (24) + : +- CometBroadcastExchange (45) + : +- CometFilter (44) + : +- CometScan parquet spark_catalog.default.item (43) + +- ReusedExchange (48) (1) Scan parquet spark_catalog.default.store_sales @@ -69,218 +78,250 @@ Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#3))] Aggregate Attributes [2]: [sum#5, count#6] Results [3]: [ss_item_sk#1, sum#7, count#8] -(6) Exchange +(6) RowToColumnar Input [3]: [ss_item_sk#1, sum#7, count#8] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(7) HashAggregate [codegen id : 2] +(7) CometColumnarExchange +Input [3]: [ss_item_sk#1, sum#7, count#8] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(8) ColumnarToRow [codegen id : 2] +Input [3]: [ss_item_sk#1, sum#7, count#8] + +(9) HashAggregate [codegen id : 2] Input [3]: [ss_item_sk#1, sum#7, count#8] Keys [1]: [ss_item_sk#1] Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#3))#9] Results [2]: [ss_item_sk#1 AS item_sk#10, cast((avg(UnscaledValue(ss_net_profit#3))#9 / 100.0) as decimal(11,6)) AS rank_col#11] -(8) Filter [codegen id : 2] +(10) Filter [codegen id : 2] Input [2]: [item_sk#10, rank_col#11] Condition : (isnotnull(rank_col#11) AND (cast(rank_col#11 as decimal(13,7)) > (0.9 * Subquery scalar-subquery#12, [id=#13]))) -(9) Sort [codegen id : 2] +(11) Sort [codegen id : 2] Input [2]: [item_sk#10, rank_col#11] Arguments: [rank_col#11 ASC NULLS FIRST], false, 0 -(10) WindowGroupLimit +(12) WindowGroupLimit Input [2]: [item_sk#10, rank_col#11] Arguments: [rank_col#11 ASC NULLS FIRST], rank(rank_col#11), 10, Partial -(11) Exchange +(13) RowToColumnar Input [2]: [item_sk#10, rank_col#11] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] -(12) Sort [codegen id : 3] +(14) CometColumnarExchange Input [2]: [item_sk#10, rank_col#11] -Arguments: [rank_col#11 ASC NULLS FIRST], false, 0 +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(15) CometSort +Input [2]: [item_sk#10, rank_col#11] +Arguments: [item_sk#10, rank_col#11], [rank_col#11 ASC NULLS FIRST] -(13) WindowGroupLimit +(16) ColumnarToRow [codegen id : 3] +Input [2]: [item_sk#10, rank_col#11] + +(17) WindowGroupLimit Input [2]: [item_sk#10, rank_col#11] Arguments: [rank_col#11 ASC NULLS FIRST], rank(rank_col#11), 10, Final -(14) Window +(18) Window Input [2]: [item_sk#10, rank_col#11] Arguments: [rank(rank_col#11) windowspecdefinition(rank_col#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#14], [rank_col#11 ASC NULLS FIRST] -(15) Filter [codegen id : 4] +(19) Filter [codegen id : 4] Input [3]: [item_sk#10, rank_col#11, rnk#14] Condition : ((rnk#14 < 11) AND isnotnull(item_sk#10)) -(16) Project [codegen id : 4] +(20) Project [codegen id : 4] Output [2]: [item_sk#10, rnk#14] Input [3]: [item_sk#10, rank_col#11, rnk#14] -(17) Exchange +(21) RowToColumnar +Input [2]: [item_sk#10, rnk#14] + +(22) CometColumnarExchange Input [2]: [item_sk#10, rnk#14] -Arguments: hashpartitioning(rnk#14, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(rnk#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(18) Sort [codegen id : 5] +(23) CometSort Input [2]: [item_sk#10, rnk#14] -Arguments: [rnk#14 ASC NULLS FIRST], false, 0 +Arguments: [item_sk#10, rnk#14], [rnk#14 ASC NULLS FIRST] -(19) ReusedExchange [Reuses operator id: 6] +(24) ReusedExchange [Reuses operator id: 7] Output [3]: [ss_item_sk#15, sum#16, count#17] -(20) HashAggregate [codegen id : 7] +(25) ColumnarToRow [codegen id : 6] +Input [3]: [ss_item_sk#15, sum#16, count#17] + +(26) HashAggregate [codegen id : 6] Input [3]: [ss_item_sk#15, sum#16, count#17] Keys [1]: [ss_item_sk#15] Functions [1]: [avg(UnscaledValue(ss_net_profit#18))] Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#18))#19] Results [2]: [ss_item_sk#15 AS item_sk#20, cast((avg(UnscaledValue(ss_net_profit#18))#19 / 100.0) as decimal(11,6)) AS rank_col#21] -(21) Filter [codegen id : 7] +(27) Filter [codegen id : 6] Input [2]: [item_sk#20, rank_col#21] Condition : (isnotnull(rank_col#21) AND (cast(rank_col#21 as decimal(13,7)) > (0.9 * ReusedSubquery Subquery scalar-subquery#12, [id=#13]))) -(22) Sort [codegen id : 7] +(28) Sort [codegen id : 6] Input [2]: [item_sk#20, rank_col#21] Arguments: [rank_col#21 DESC NULLS LAST], false, 0 -(23) WindowGroupLimit +(29) WindowGroupLimit Input [2]: [item_sk#20, rank_col#21] Arguments: [rank_col#21 DESC NULLS LAST], rank(rank_col#21), 10, Partial -(24) Exchange +(30) RowToColumnar Input [2]: [item_sk#20, rank_col#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] -(25) Sort [codegen id : 8] +(31) CometColumnarExchange Input [2]: [item_sk#20, rank_col#21] -Arguments: [rank_col#21 DESC NULLS LAST], false, 0 +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(26) WindowGroupLimit +(32) CometSort +Input [2]: [item_sk#20, rank_col#21] +Arguments: [item_sk#20, rank_col#21], [rank_col#21 DESC NULLS LAST] + +(33) ColumnarToRow [codegen id : 7] +Input [2]: [item_sk#20, rank_col#21] + +(34) WindowGroupLimit Input [2]: [item_sk#20, rank_col#21] Arguments: [rank_col#21 DESC NULLS LAST], rank(rank_col#21), 10, Final -(27) Window +(35) Window Input [2]: [item_sk#20, rank_col#21] Arguments: [rank(rank_col#21) windowspecdefinition(rank_col#21 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#22], [rank_col#21 DESC NULLS LAST] -(28) Filter [codegen id : 9] +(36) Filter [codegen id : 8] Input [3]: [item_sk#20, rank_col#21, rnk#22] Condition : ((rnk#22 < 11) AND isnotnull(item_sk#20)) -(29) Project [codegen id : 9] +(37) Project [codegen id : 8] Output [2]: [item_sk#20, rnk#22] Input [3]: [item_sk#20, rank_col#21, rnk#22] -(30) Exchange +(38) RowToColumnar Input [2]: [item_sk#20, rnk#22] -Arguments: hashpartitioning(rnk#22, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(31) Sort [codegen id : 10] +(39) CometColumnarExchange Input [2]: [item_sk#20, rnk#22] -Arguments: [rnk#22 ASC NULLS FIRST], false, 0 +Arguments: hashpartitioning(rnk#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(32) SortMergeJoin [codegen id : 13] -Left keys [1]: [rnk#14] -Right keys [1]: [rnk#22] -Join type: Inner -Join condition: None +(40) CometSort +Input [2]: [item_sk#20, rnk#22] +Arguments: [item_sk#20, rnk#22], [rnk#22 ASC NULLS FIRST] + +(41) CometSortMergeJoin +Left output [2]: [item_sk#10, rnk#14] +Right output [2]: [item_sk#20, rnk#22] +Arguments: [rnk#14], [rnk#22], Inner -(33) Project [codegen id : 13] -Output [3]: [item_sk#10, rnk#14, item_sk#20] +(42) CometProject Input [4]: [item_sk#10, rnk#14, item_sk#20, rnk#22] +Arguments: [item_sk#10, rnk#14, item_sk#20], [item_sk#10, rnk#14, item_sk#20] -(34) Scan parquet spark_catalog.default.item +(43) Scan parquet spark_catalog.default.item Output [2]: [i_item_sk#23, i_product_name#24] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(35) CometFilter +(44) CometFilter Input [2]: [i_item_sk#23, i_product_name#24] Condition : isnotnull(i_item_sk#23) -(36) ColumnarToRow [codegen id : 11] -Input [2]: [i_item_sk#23, i_product_name#24] - -(37) BroadcastExchange +(45) CometBroadcastExchange Input [2]: [i_item_sk#23, i_product_name#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +Arguments: [i_item_sk#23, i_product_name#24] -(38) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [item_sk#10] -Right keys [1]: [i_item_sk#23] -Join type: Inner -Join condition: None +(46) CometBroadcastHashJoin +Left output [3]: [item_sk#10, rnk#14, item_sk#20] +Right output [2]: [i_item_sk#23, i_product_name#24] +Arguments: [item_sk#10], [i_item_sk#23], Inner, BuildRight -(39) Project [codegen id : 13] -Output [3]: [rnk#14, item_sk#20, i_product_name#24] +(47) CometProject Input [5]: [item_sk#10, rnk#14, item_sk#20, i_item_sk#23, i_product_name#24] +Arguments: [rnk#14, item_sk#20, i_product_name#24], [rnk#14, item_sk#20, i_product_name#24] -(40) ReusedExchange [Reuses operator id: 37] +(48) ReusedExchange [Reuses operator id: 45] Output [2]: [i_item_sk#25, i_product_name#26] -(41) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [item_sk#20] -Right keys [1]: [i_item_sk#25] -Join type: Inner -Join condition: None +(49) CometBroadcastHashJoin +Left output [3]: [rnk#14, item_sk#20, i_product_name#24] +Right output [2]: [i_item_sk#25, i_product_name#26] +Arguments: [item_sk#20], [i_item_sk#25], Inner, BuildRight -(42) Project [codegen id : 13] -Output [3]: [rnk#14, i_product_name#24 AS best_performing#27, i_product_name#26 AS worst_performing#28] +(50) CometProject Input [5]: [rnk#14, item_sk#20, i_product_name#24, i_item_sk#25, i_product_name#26] +Arguments: [rnk#14, best_performing#27, worst_performing#28], [rnk#14, i_product_name#24 AS best_performing#27, i_product_name#26 AS worst_performing#28] + +(51) CometTakeOrderedAndProject +Input [3]: [rnk#14, best_performing#27, worst_performing#28] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[rnk#14 ASC NULLS FIRST], output=[rnk#14,best_performing#27,worst_performing#28]), [rnk#14, best_performing#27, worst_performing#28], 100, [rnk#14 ASC NULLS FIRST], [rnk#14, best_performing#27, worst_performing#28] -(43) TakeOrderedAndProject +(52) ColumnarToRow [codegen id : 9] Input [3]: [rnk#14, best_performing#27, worst_performing#28] -Arguments: 100, [rnk#14 ASC NULLS FIRST], [rnk#14, best_performing#27, worst_performing#28] ===== Subqueries ===== -Subquery:1 Hosting operator id = 8 Hosting Expression = Subquery scalar-subquery#12, [id=#13] -* HashAggregate (50) -+- Exchange (49) - +- * HashAggregate (48) - +- * ColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan parquet spark_catalog.default.store_sales (44) +Subquery:1 Hosting operator id = 10 Hosting Expression = Subquery scalar-subquery#12, [id=#13] +* HashAggregate (61) ++- * ColumnarToRow (60) + +- CometColumnarExchange (59) + +- RowToColumnar (58) + +- * HashAggregate (57) + +- * ColumnarToRow (56) + +- CometProject (55) + +- CometFilter (54) + +- CometScan parquet spark_catalog.default.store_sales (53) -(44) Scan parquet spark_catalog.default.store_sales +(53) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_addr_sk#29, ss_store_sk#30, ss_net_profit#31, ss_sold_date_sk#32] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)] ReadSchema: struct -(45) CometFilter +(54) CometFilter Input [4]: [ss_addr_sk#29, ss_store_sk#30, ss_net_profit#31, ss_sold_date_sk#32] Condition : ((isnotnull(ss_store_sk#30) AND (ss_store_sk#30 = 4)) AND isnull(ss_addr_sk#29)) -(46) CometProject +(55) CometProject Input [4]: [ss_addr_sk#29, ss_store_sk#30, ss_net_profit#31, ss_sold_date_sk#32] Arguments: [ss_store_sk#30, ss_net_profit#31], [ss_store_sk#30, ss_net_profit#31] -(47) ColumnarToRow [codegen id : 1] +(56) ColumnarToRow [codegen id : 1] Input [2]: [ss_store_sk#30, ss_net_profit#31] -(48) HashAggregate [codegen id : 1] +(57) HashAggregate [codegen id : 1] Input [2]: [ss_store_sk#30, ss_net_profit#31] Keys [1]: [ss_store_sk#30] Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#31))] Aggregate Attributes [2]: [sum#33, count#34] Results [3]: [ss_store_sk#30, sum#35, count#36] -(49) Exchange +(58) RowToColumnar +Input [3]: [ss_store_sk#30, sum#35, count#36] + +(59) CometColumnarExchange +Input [3]: [ss_store_sk#30, sum#35, count#36] +Arguments: hashpartitioning(ss_store_sk#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(60) ColumnarToRow [codegen id : 2] Input [3]: [ss_store_sk#30, sum#35, count#36] -Arguments: hashpartitioning(ss_store_sk#30, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(50) HashAggregate [codegen id : 2] +(61) HashAggregate [codegen id : 2] Input [3]: [ss_store_sk#30, sum#35, count#36] Keys [1]: [ss_store_sk#30] Functions [1]: [avg(UnscaledValue(ss_net_profit#31))] Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#31))#37] Results [1]: [cast((avg(UnscaledValue(ss_net_profit#31))#37 / 100.0) as decimal(11,6)) AS rank_col#38] -Subquery:2 Hosting operator id = 21 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] +Subquery:2 Hosting operator id = 27 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/simplified.txt index 12c8e0af9..9794c9374 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/simplified.txt @@ -1,16 +1,16 @@ -TakeOrderedAndProject [rnk,best_performing,worst_performing] - WholeStageCodegen (13) - Project [rnk,i_product_name,i_product_name] - BroadcastHashJoin [item_sk,i_item_sk] - Project [rnk,item_sk,i_product_name] - BroadcastHashJoin [item_sk,i_item_sk] - Project [item_sk,rnk,item_sk] - SortMergeJoin [rnk,rnk] - InputAdapter - WholeStageCodegen (5) - Sort [rnk] - InputAdapter - Exchange [rnk] #1 +WholeStageCodegen (9) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [rnk,best_performing,worst_performing] + CometProject [i_product_name,i_product_name] [rnk,best_performing,worst_performing] + CometBroadcastHashJoin [rnk,item_sk,i_product_name,i_item_sk,i_product_name] + CometProject [rnk,item_sk,i_product_name] + CometBroadcastHashJoin [item_sk,rnk,item_sk,i_item_sk,i_product_name] + CometProject [item_sk,rnk,item_sk] + CometSortMergeJoin [item_sk,rnk,item_sk,rnk] + CometSort [item_sk,rnk] + CometColumnarExchange [rnk] #1 + RowToColumnar WholeStageCodegen (4) Project [item_sk,rnk] Filter [rnk,item_sk] @@ -18,64 +18,66 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] Window [rank_col] WindowGroupLimit [rank_col] WholeStageCodegen (3) - Sort [rank_col] + ColumnarToRow InputAdapter - Exchange #2 - WindowGroupLimit [rank_col] - WholeStageCodegen (2) - Sort [rank_col] - Filter [rank_col] - Subquery #1 - WholeStageCodegen (2) - HashAggregate [ss_store_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),rank_col,sum,count] - InputAdapter - Exchange [ss_store_sk] #4 - WholeStageCodegen (1) - HashAggregate [ss_store_sk,ss_net_profit] [sum,count,sum,count] - ColumnarToRow - InputAdapter - CometProject [ss_store_sk,ss_net_profit] - CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,sum,count] - InputAdapter - Exchange [ss_item_sk] #3 - WholeStageCodegen (1) - HashAggregate [ss_item_sk,ss_net_profit] [sum,count,sum,count] + CometSort [item_sk,rank_col] + CometColumnarExchange #2 + RowToColumnar + WindowGroupLimit [rank_col] + WholeStageCodegen (2) + Sort [rank_col] + Filter [rank_col] + Subquery #1 + WholeStageCodegen (2) + HashAggregate [ss_store_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),rank_col,sum,count] ColumnarToRow InputAdapter - CometProject [ss_item_sk,ss_net_profit] - CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - InputAdapter - WholeStageCodegen (10) - Sort [rnk] - InputAdapter - Exchange [rnk] #5 - WholeStageCodegen (9) + CometColumnarExchange [ss_store_sk] #4 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [ss_store_sk,ss_net_profit] [sum,count,sum,count] + ColumnarToRow + InputAdapter + CometProject [ss_store_sk,ss_net_profit] + CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,sum,count] + ColumnarToRow + InputAdapter + CometColumnarExchange [ss_item_sk] #3 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [ss_item_sk,ss_net_profit] [sum,count,sum,count] + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_net_profit] + CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometSort [item_sk,rnk] + CometColumnarExchange [rnk] #5 + RowToColumnar + WholeStageCodegen (8) Project [item_sk,rnk] Filter [rnk,item_sk] InputAdapter Window [rank_col] WindowGroupLimit [rank_col] - WholeStageCodegen (8) - Sort [rank_col] + WholeStageCodegen (7) + ColumnarToRow InputAdapter - Exchange #6 - WindowGroupLimit [rank_col] - WholeStageCodegen (7) - Sort [rank_col] - Filter [rank_col] - ReusedSubquery [rank_col] #1 - HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,sum,count] - InputAdapter - ReusedExchange [ss_item_sk,sum,count] #3 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_product_name] - CometScan parquet spark_catalog.default.item [i_item_sk,i_product_name] - InputAdapter - ReusedExchange [i_item_sk,i_product_name] #7 + CometSort [item_sk,rank_col] + CometColumnarExchange #6 + RowToColumnar + WindowGroupLimit [rank_col] + WholeStageCodegen (6) + Sort [rank_col] + Filter [rank_col] + ReusedSubquery [rank_col] #1 + HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,sum,count] + ColumnarToRow + InputAdapter + ReusedExchange [ss_item_sk,sum,count] #3 + CometBroadcastExchange [i_item_sk,i_product_name] #7 + CometFilter [i_item_sk,i_product_name] + CometScan parquet spark_catalog.default.item [i_item_sk,i_product_name] + ReusedExchange [i_item_sk,i_product_name] #7 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 f128499e3..044247540 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 @@ -1,40 +1,42 @@ == Physical Plan == -TakeOrderedAndProject (36) -+- * HashAggregate (35) - +- Exchange (34) - +- * HashAggregate (33) - +- * Project (32) - +- * Filter (31) - +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (30) - :- * 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) - +- * ColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan parquet spark_catalog.default.item (25) +TakeOrderedAndProject (38) ++- * HashAggregate (37) + +- * ColumnarToRow (36) + +- CometColumnarExchange (35) + +- RowToColumnar (34) + +- * HashAggregate (33) + +- * Project (32) + +- * Filter (31) + +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (30) + :- * 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) + +- * ColumnarToRow (28) + +- CometProject (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.item (25) (1) Scan parquet spark_catalog.default.web_sales @@ -195,50 +197,56 @@ Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] Aggregate Attributes [1]: [sum#19] Results [3]: [ca_zip#11, ca_city#10, sum#20] -(34) Exchange +(34) RowToColumnar 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 : 3] +(35) CometColumnarExchange +Input [3]: [ca_zip#11, ca_city#10, sum#20] +Arguments: hashpartitioning(ca_zip#11, ca_city#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(36) ColumnarToRow [codegen id : 3] +Input [3]: [ca_zip#11, ca_city#10, sum#20] + +(37) 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))#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 +(38) TakeOrderedAndProject 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 ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (41) -+- * ColumnarToRow (40) - +- CometProject (39) - +- CometFilter (38) - +- CometScan parquet spark_catalog.default.date_dim (37) +BroadcastExchange (43) ++- * ColumnarToRow (42) + +- CometProject (41) + +- CometFilter (40) + +- CometScan parquet spark_catalog.default.date_dim (39) -(37) Scan parquet spark_catalog.default.date_dim +(39) 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 -(38) CometFilter +(40) 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)) -(39) CometProject +(41) CometProject 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] +(42) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#12] -(41) BroadcastExchange +(43) BroadcastExchange Input [1]: [d_date_sk#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/q45/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/simplified.txt index f60fdb18a..8550b91c6 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,51 +1,53 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] 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 (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] - ColumnarToRow - InputAdapter - CometProject [ws_sales_price,ca_city,ca_zip,i_item_id] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ca_city,ca_zip,i_item_sk,i_item_id] - CometProject [ws_item_sk,ws_sales_price,ca_city,ca_zip] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip,d_date_sk] - CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_city,ca_zip] - CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk,c_customer_sk,c_current_addr_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_city,ca_zip] #4 - CometFilter [ca_address_sk,ca_city,ca_zip] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + ColumnarToRow + InputAdapter + CometColumnarExchange [ca_zip,ca_city] #1 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometProject [ws_sales_price,ca_city,ca_zip,i_item_id] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ca_city,ca_zip,i_item_sk,i_item_id] + CometProject [ws_item_sk,ws_sales_price,ca_city,ca_zip] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip,d_date_sk] + CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_city,ca_zip] + CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk,c_customer_sk,c_current_addr_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange [ca_address_sk,ca_city,ca_zip] #4 + CometFilter [ca_address_sk,ca_city,ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [i_item_id] + CometFilter [i_item_sk,i_item_id] + 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/q46/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/explain.txt index 71f97587a..549b421d7 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,46 +1,48 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * Project (41) - +- * BroadcastHashJoin Inner BuildRight (40) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * HashAggregate (29) - : : +- Exchange (28) - : : +- * HashAggregate (27) - : : +- * 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) - +- BroadcastExchange (39) - +- * ColumnarToRow (38) - +- CometFilter (37) - +- CometScan parquet spark_catalog.default.customer_address (36) +TakeOrderedAndProject (44) ++- * Project (43) + +- * BroadcastHashJoin Inner BuildRight (42) + :- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * HashAggregate (31) + : : +- * ColumnarToRow (30) + : : +- CometColumnarExchange (29) + : : +- RowToColumnar (28) + : : +- * HashAggregate (27) + : : +- * 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 (35) + : +- * ColumnarToRow (34) + : +- CometFilter (33) + : +- CometScan parquet spark_catalog.default.customer (32) + +- BroadcastExchange (41) + +- * ColumnarToRow (40) + +- CometFilter (39) + +- CometScan parquet spark_catalog.default.customer_address (38) (1) Scan parquet spark_catalog.default.store_sales @@ -173,106 +175,112 @@ Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(Unscale 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 +(28) RowToColumnar 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 : 4] +(29) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(30) ColumnarToRow [codegen id : 4] +Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, sum#22, sum#23] + +(31) 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))#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 +(32) Scan parquet spark_catalog.default.customer 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 +(33) CometFilter 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 : 2] +(34) 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 +(35) BroadcastExchange 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 : 4] +(36) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_customer_sk#1] Right keys [1]: [c_customer_sk#29] Join type: Inner Join condition: None -(35) Project [codegen id : 4] +(37) 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 +(38) 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 -(37) CometFilter +(39) 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] +(40) ColumnarToRow [codegen id : 3] Input [2]: [ca_address_sk#33, ca_city#34] -(39) BroadcastExchange +(41) BroadcastExchange Input [2]: [ca_address_sk#33, ca_city#34] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(40) BroadcastHashJoin [codegen id : 4] +(42) 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#34 = bought_city#26) -(41) Project [codegen id : 4] +(43) 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] -(42) TakeOrderedAndProject +(44) 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 (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 +(45) 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 -(44) CometFilter +(46) 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)) -(45) CometProject +(47) CometProject Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(46) ColumnarToRow [codegen id : 1] +(48) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(47) BroadcastExchange +(49) BroadcastExchange Input [1]: [d_date_sk#10] 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 9ce3a98eb..794e4979c 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 @@ -5,45 +5,47 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu Project [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name] BroadcastHashJoin [ss_customer_sk,c_customer_sk] 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 (1) - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] - ColumnarToRow - InputAdapter - CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] - CometBroadcastHashJoin [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_address_sk,ca_city] - CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,hd_demo_sk] - CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,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_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [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] - 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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_city] - CometScan parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange [hd_demo_sk] #5 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [ca_address_sk,ca_city] #6 - CometFilter [ca_address_sk,ca_city] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] + ColumnarToRow + InputAdapter + CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] + ColumnarToRow + InputAdapter + CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] + CometBroadcastHashJoin [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_address_sk,ca_city] + CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,hd_demo_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,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_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [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] + 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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometBroadcastExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometBroadcastExchange [s_store_sk] #4 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_city] + CometScan parquet spark_catalog.default.store [s_store_sk,s_city] + CometBroadcastExchange [hd_demo_sk] #5 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange [ca_address_sk,ca_city] #6 + CometFilter [ca_address_sk,ca_city] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) 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 12f171f31..355f13a9a 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 @@ -1,49 +1,57 @@ == Physical Plan == -TakeOrderedAndProject (45) -+- * Project (44) - +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (28) - : : +- * Filter (27) - : : +- Window (26) - : : +- * Filter (25) - : : +- Window (24) - : : +- * Sort (23) - : : +- Exchange (22) - : : +- * HashAggregate (21) - : : +- Exchange (20) - : : +- * HashAggregate (19) - : : +- * 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) - : +- * Project (34) - : +- Window (33) - : +- * Sort (32) - : +- Exchange (31) - : +- * HashAggregate (30) - : +- ReusedExchange (29) - +- BroadcastExchange (42) - +- * Project (41) - +- Window (40) - +- * Sort (39) - +- ReusedExchange (38) +TakeOrderedAndProject (53) ++- * Project (52) + +- * BroadcastHashJoin Inner BuildRight (51) + :- * Project (44) + : +- * BroadcastHashJoin Inner BuildRight (43) + : :- * Project (32) + : : +- * Filter (31) + : : +- Window (30) + : : +- * Filter (29) + : : +- Window (28) + : : +- * ColumnarToRow (27) + : : +- CometSort (26) + : : +- CometColumnarExchange (25) + : : +- RowToColumnar (24) + : : +- * HashAggregate (23) + : : +- * ColumnarToRow (22) + : : +- CometColumnarExchange (21) + : : +- RowToColumnar (20) + : : +- * HashAggregate (19) + : : +- * 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 (42) + : +- * Project (41) + : +- Window (40) + : +- * ColumnarToRow (39) + : +- CometSort (38) + : +- CometColumnarExchange (37) + : +- RowToColumnar (36) + : +- * HashAggregate (35) + : +- * ColumnarToRow (34) + : +- ReusedExchange (33) + +- BroadcastExchange (50) + +- * Project (49) + +- Window (48) + +- * ColumnarToRow (47) + +- CometSort (46) + +- ReusedExchange (45) (1) Scan parquet spark_catalog.default.item @@ -140,142 +148,166 @@ 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] -(20) Exchange +(20) RowToColumnar 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=1] -(21) HashAggregate [codegen id : 2] +(21) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(22) ColumnarToRow [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] + +(23) 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))] 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] -(22) Exchange +(24) RowToColumnar +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] + +(25) CometColumnarExchange 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=2] +Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(23) Sort [codegen id : 3] +(26) CometSort 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 +Arguments: [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], [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] -(24) Window +(27) ColumnarToRow [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] + +(28) 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] -(25) Filter [codegen id : 4] +(29) 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)) -(26) Window +(30) 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] -(27) Filter [codegen id : 13] +(31) 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 : 13] +(32) 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] +(33) ReusedExchange [Reuses operator id: 21] 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 : 6] +(34) ColumnarToRow [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] + +(35) 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))] 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#30] -(31) Exchange +(36) RowToColumnar +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] + +(37) CometColumnarExchange 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=3] +Arguments: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(32) Sort [codegen id : 7] +(38) CometSort 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 +Arguments: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30], [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] -(33) Window +(39) ColumnarToRow [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] + +(40) 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#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 : 8] +(41) 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 +(42) 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=4] -(36) BroadcastHashJoin [codegen id : 13] +(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#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 : 13] +(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, 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] +(45) ReusedExchange [Reuses operator id: 37] 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 : 11] +(46) CometSort 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 +Arguments: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38], [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] -(40) Window +(47) ColumnarToRow [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] + +(48) 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#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 : 12] +(49) 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 +(50) 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=5] -(43) BroadcastHashJoin [codegen id : 13] +(51) 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 : 13] +(52) 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] -(45) TakeOrderedAndProject +(53) 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] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (49) -+- * ColumnarToRow (48) - +- CometFilter (47) - +- CometScan parquet spark_catalog.default.date_dim (46) +BroadcastExchange (57) ++- * ColumnarToRow (56) + +- CometFilter (55) + +- CometScan parquet spark_catalog.default.date_dim (54) -(46) Scan parquet spark_catalog.default.date_dim +(54) 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 -(47) CometFilter +(55) 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)) -(48) ColumnarToRow [codegen id : 1] +(56) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -(49) BroadcastExchange +(57) 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=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 1db8e7528..4c9474631 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 @@ -13,41 +13,45 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (3) - Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + ColumnarToRow InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name] #1 - 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 (1) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometFilter [i_item_sk,i_brand,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #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] + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #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 #7 WholeStageCodegen (8) @@ -55,13 +59,16 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (7) - Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + ColumnarToRow InputAdapter - 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 + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #8 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 InputAdapter BroadcastExchange #9 WholeStageCodegen (12) @@ -69,6 +76,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (11) - Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + ColumnarToRow InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + 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 5197d99f8..9b30bfe48 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 @@ -1,32 +1,34 @@ == Physical Plan == -* 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.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) +* HashAggregate (30) ++- * ColumnarToRow (29) + +- CometColumnarExchange (28) + +- RowToColumnar (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.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 @@ -155,11 +157,17 @@ Functions [1]: [partial_sum(ss_quantity#4)] Aggregate Attributes [1]: [sum#18] Results [1]: [sum#19] -(27) Exchange +(27) RowToColumnar Input [1]: [sum#19] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] -(28) HashAggregate [codegen id : 2] +(28) CometColumnarExchange +Input [1]: [sum#19] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(29) ColumnarToRow [codegen id : 2] +Input [1]: [sum#19] + +(30) HashAggregate [codegen id : 2] Input [1]: [sum#19] Keys: [] Functions [1]: [sum(ss_quantity#4)] @@ -169,32 +177,32 @@ Results [1]: [sum(ss_quantity#4)#20 AS sum(ss_quantity)#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (33) -+- * ColumnarToRow (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan parquet spark_catalog.default.date_dim (29) +BroadcastExchange (35) ++- * ColumnarToRow (34) + +- CometProject (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.date_dim (31) -(29) Scan parquet spark_catalog.default.date_dim +(31) 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 -(30) CometFilter +(32) 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)) -(31) CometProject +(33) CometProject Input [2]: [d_date_sk#16, d_year#17] Arguments: [d_date_sk#16], [d_date_sk#16] -(32) ColumnarToRow [codegen id : 1] +(34) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#16] -(33) BroadcastExchange +(35) BroadcastExchange Input [1]: [d_date_sk#16] 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 23e083535..e1595f3e7 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,40 +1,42 @@ WholeStageCodegen (2) HashAggregate [sum] [sum(ss_quantity),sum(ss_quantity),sum] - InputAdapter - Exchange #1 - WholeStageCodegen (1) - HashAggregate [ss_quantity] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [ss_quantity] - CometBroadcastHashJoin [ss_quantity,ss_sold_date_sk,d_date_sk] - CometProject [ss_quantity,ss_sold_date_sk] - CometBroadcastHashJoin [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk,ca_address_sk,ca_state] - CometProject [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,cd_demo_sk,cd_marital_status,cd_education_status] - CometProject [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,s_store_sk] - CometFilter [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] - 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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk] #3 - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #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 [ca_address_sk,ca_state] #5 - CometProject [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ColumnarToRow + InputAdapter + CometColumnarExchange #1 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [ss_quantity] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [ss_quantity] + CometBroadcastHashJoin [ss_quantity,ss_sold_date_sk,d_date_sk] + CometProject [ss_quantity,ss_sold_date_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk,ca_address_sk,ca_state] + CometProject [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk] + CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,cd_demo_sk,cd_marital_status,cd_education_status] + CometProject [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] + CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,s_store_sk] + CometFilter [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] + 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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [s_store_sk] #3 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #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 [ca_address_sk,ca_state] #5 + CometProject [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + 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 124a22ef9..f8e0c93d7 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,81 +1,95 @@ == Physical Plan == -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) +* ColumnarToRow (91) ++- CometTakeOrderedAndProject (90) + +- CometHashAggregate (89) + +- CometColumnarExchange (88) + +- RowToColumnar (87) + +- * HashAggregate (86) + +- Union (85) + :- * Project (30) + : +- * Filter (29) + : +- Window (28) + : +- * Sort (27) + : +- Window (26) + : +- * ColumnarToRow (25) + : +- CometSort (24) + : +- CometColumnarExchange (23) + : +- RowToColumnar (22) + : +- * HashAggregate (21) + : +- * ColumnarToRow (20) + : +- CometColumnarExchange (19) + : +- RowToColumnar (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 (57) + : +- * Filter (56) + : +- Window (55) + : +- * Sort (54) + : +- Window (53) + : +- * ColumnarToRow (52) + : +- CometSort (51) + : +- CometColumnarExchange (50) + : +- RowToColumnar (49) + : +- * HashAggregate (48) + : +- * ColumnarToRow (47) + : +- CometColumnarExchange (46) + : +- RowToColumnar (45) + : +- * HashAggregate (44) + : +- * ColumnarToRow (43) + : +- CometProject (42) + : +- CometBroadcastHashJoin (41) + : :- CometProject (39) + : : +- CometBroadcastHashJoin (38) + : : :- CometBroadcastExchange (34) + : : : +- CometProject (33) + : : : +- CometFilter (32) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (31) + : : +- CometProject (37) + : : +- CometFilter (36) + : : +- CometScan parquet spark_catalog.default.catalog_returns (35) + : +- ReusedExchange (40) + +- * Project (84) + +- * Filter (83) + +- Window (82) + +- * Sort (81) + +- Window (80) + +- * ColumnarToRow (79) + +- CometSort (78) + +- CometColumnarExchange (77) + +- RowToColumnar (76) + +- * HashAggregate (75) + +- * ColumnarToRow (74) + +- CometColumnarExchange (73) + +- RowToColumnar (72) + +- * HashAggregate (71) + +- * ColumnarToRow (70) + +- CometProject (69) + +- CometBroadcastHashJoin (68) + :- CometProject (66) + : +- CometBroadcastHashJoin (65) + : :- CometBroadcastExchange (61) + : : +- CometProject (60) + : : +- CometFilter (59) + : : +- CometScan parquet spark_catalog.default.store_sales (58) + : +- CometProject (64) + : +- CometFilter (63) + : +- CometScan parquet spark_catalog.default.store_returns (62) + +- ReusedExchange (67) (1) Scan parquet spark_catalog.default.web_sales @@ -160,46 +174,58 @@ Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coa 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] -(18) Exchange +(18) RowToColumnar 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] -(19) HashAggregate [codegen id : 2] +(19) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(20) ColumnarToRow [codegen id : 2] +Input [7]: [ws_item_sk#1, sum#22, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] + +(21) 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))#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] -(20) Exchange +(22) RowToColumnar Input [3]: [item#32, return_ratio#33, currency_ratio#34] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] -(21) Sort [codegen id : 3] +(23) CometColumnarExchange Input [3]: [item#32, return_ratio#33, currency_ratio#34] -Arguments: [return_ratio#33 ASC NULLS FIRST], false, 0 +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(22) Window +(24) CometSort +Input [3]: [item#32, return_ratio#33, currency_ratio#34] +Arguments: [item#32, return_ratio#33, currency_ratio#34], [return_ratio#33 ASC NULLS FIRST] + +(25) ColumnarToRow [codegen id : 3] +Input [3]: [item#32, return_ratio#33, currency_ratio#34] + +(26) 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] -(23) Sort [codegen id : 4] +(27) 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 -(24) Window +(28) 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] -(25) Filter [codegen id : 5] +(29) 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)) -(26) Project [codegen id : 5] +(30) 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] -(27) Scan parquet spark_catalog.default.catalog_sales +(31) 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 [] @@ -207,104 +233,116 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_so 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 -(28) CometFilter +(32) 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)) -(29) CometProject +(33) 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] -(30) CometBroadcastExchange +(34) 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] -(31) Scan parquet spark_catalog.default.catalog_returns +(35) 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 -(32) CometFilter +(36) 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 +(37) 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 +(38) 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 +(39) 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] +(40) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#50] -(37) CometBroadcastHashJoin +(41) 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 +(42) 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] +(43) 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] +(44) 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 +(45) RowToColumnar +Input [7]: [cs_item_sk#38, sum#57, sum#58, sum#59, isEmpty#60, sum#61, isEmpty#62] + +(46) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=3] + +(47) ColumnarToRow [codegen id : 7] 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] +(48) 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 +(49) RowToColumnar +Input [3]: [item#67, return_ratio#68, currency_ratio#69] + +(50) CometColumnarExchange +Input [3]: [item#67, return_ratio#68, currency_ratio#69] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(51) CometSort Input [3]: [item#67, return_ratio#68, currency_ratio#69] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: [item#67, return_ratio#68, currency_ratio#69], [return_ratio#68 ASC NULLS FIRST] -(44) Sort [codegen id : 8] +(52) ColumnarToRow [codegen id : 8] Input [3]: [item#67, return_ratio#68, currency_ratio#69] -Arguments: [return_ratio#68 ASC NULLS FIRST], false, 0 -(45) Window +(53) 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] -(46) Sort [codegen id : 9] +(54) 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 -(47) Window +(55) 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] -(48) Filter [codegen id : 10] +(56) 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)) -(49) Project [codegen id : 10] +(57) 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] -(50) Scan parquet spark_catalog.default.store_sales +(58) 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 [] @@ -312,161 +350,177 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#78), dynamicpruningexpression(ss_so 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 -(51) CometFilter +(59) 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)) -(52) CometProject +(60) 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] -(53) CometBroadcastExchange +(61) 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] -(54) Scan parquet spark_catalog.default.store_returns +(62) 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 -(55) CometFilter +(63) 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 +(64) 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 +(65) 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 +(66) 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] +(67) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#85] -(60) CometBroadcastHashJoin +(68) 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 +(69) 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] +(70) 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] +(71) 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 +(72) RowToColumnar +Input [7]: [ss_item_sk#73, sum#92, sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] + +(73) CometColumnarExchange 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] +Arguments: hashpartitioning(ss_item_sk#73, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(65) HashAggregate [codegen id : 12] +(74) ColumnarToRow [codegen id : 12] +Input [7]: [ss_item_sk#73, sum#92, sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] + +(75) 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 +(76) RowToColumnar +Input [3]: [item#102, return_ratio#103, currency_ratio#104] + +(77) CometColumnarExchange Input [3]: [item#102, return_ratio#103, currency_ratio#104] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(67) Sort [codegen id : 13] +(78) CometSort Input [3]: [item#102, return_ratio#103, currency_ratio#104] -Arguments: [return_ratio#103 ASC NULLS FIRST], false, 0 +Arguments: [item#102, return_ratio#103, currency_ratio#104], [return_ratio#103 ASC NULLS FIRST] -(68) Window +(79) ColumnarToRow [codegen id : 13] +Input [3]: [item#102, return_ratio#103, currency_ratio#104] + +(80) 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] -(69) Sort [codegen id : 14] +(81) 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 -(70) Window +(82) 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] -(71) Filter [codegen id : 15] +(83) 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)) -(72) Project [codegen id : 15] +(84) 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] -(73) Union +(85) Union -(74) HashAggregate [codegen id : 16] +(86) 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#37, item#32, return_ratio#33, return_rank#35, currency_rank#36] -(75) Exchange +(87) RowToColumnar +Input [5]: [channel#37, item#32, return_ratio#33, return_rank#35, currency_rank#36] + +(88) CometColumnarExchange 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] +Arguments: hashpartitioning(channel#37, item#32, return_ratio#33, return_rank#35, currency_rank#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(76) HashAggregate [codegen id : 17] +(89) CometHashAggregate 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#37, item#32, return_ratio#33, return_rank#35, currency_rank#36] -(77) TakeOrderedAndProject +(90) CometTakeOrderedAndProject +Input [5]: [channel#37, item#32, return_ratio#33, return_rank#35, currency_rank#36] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#37 ASC NULLS FIRST,return_rank#35 ASC NULLS FIRST,currency_rank#36 ASC NULLS FIRST], output=[channel#37,item#32,return_ratio#33,return_rank#35,currency_rank#36]), [channel#37, item#32, return_ratio#33, return_rank#35, currency_rank#36], 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] + +(91) ColumnarToRow [codegen id : 17] 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 (82) -+- * ColumnarToRow (81) - +- CometProject (80) - +- CometFilter (79) - +- CometScan parquet spark_catalog.default.date_dim (78) +BroadcastExchange (96) ++- * ColumnarToRow (95) + +- CometProject (94) + +- CometFilter (93) + +- CometScan parquet spark_catalog.default.date_dim (92) -(78) Scan parquet spark_catalog.default.date_dim +(92) 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 -(79) CometFilter +(93) 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)) -(80) CometProject +(94) CometProject Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(81) ColumnarToRow [codegen id : 1] +(95) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(82) BroadcastExchange +(96) 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 = 27 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 31 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#78 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 58 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 bd3b21cdb..4d4e531a0 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,121 +1,135 @@ -TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] - WholeStageCodegen (17) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Exchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (16) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (5) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (4) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (3) - Sort [return_ratio] - InputAdapter - Exchange #2 - 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 (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] +WholeStageCodegen (17) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] + CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] + CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 + RowToColumnar + WholeStageCodegen (16) + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] + InputAdapter + Union + WholeStageCodegen (5) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (4) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #2 + RowToColumnar + 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] ColumnarToRow InputAdapter - CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - 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 (9) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (8) - Sort [return_ratio] - InputAdapter - 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] #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] + CometColumnarExchange [ws_item_sk] #3 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + 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 (9) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #7 + RowToColumnar + 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] ColumnarToRow InputAdapter - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_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] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - 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 (14) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (13) - Sort [return_ratio] - InputAdapter - 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] #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] + CometColumnarExchange [cs_item_sk] #8 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_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] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + 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 (14) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (13) + ColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #10 + RowToColumnar + 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] ColumnarToRow InputAdapter - CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_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] - ReusedSubquery [d_date_sk] #1 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - ReusedExchange [d_date_sk] #6 + CometColumnarExchange [ss_item_sk] #11 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_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] + ReusedSubquery [d_date_sk] #1 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + 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 e6d836892..905c7bd88 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 @@ -1,74 +1,82 @@ == Physical Plan == -TakeOrderedAndProject (70) -+- * HashAggregate (69) - +- Exchange (68) - +- * HashAggregate (67) - +- * Expand (66) - +- Union (65) - :- * 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) - +- * 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) +TakeOrderedAndProject (78) ++- * HashAggregate (77) + +- * ColumnarToRow (76) + +- CometColumnarExchange (75) + +- RowToColumnar (74) + +- * HashAggregate (73) + +- * Expand (72) + +- Union (71) + :- * HashAggregate (24) + : +- * ColumnarToRow (23) + : +- CometColumnarExchange (22) + : +- RowToColumnar (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 (45) + : +- * ColumnarToRow (44) + : +- CometColumnarExchange (43) + : +- RowToColumnar (42) + : +- * HashAggregate (41) + : +- * ColumnarToRow (40) + : +- CometProject (39) + : +- CometBroadcastHashJoin (38) + : :- CometProject (34) + : : +- CometBroadcastHashJoin (33) + : : :- CometUnion (31) + : : : :- CometProject (27) + : : : : +- CometFilter (26) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (25) + : : : +- CometProject (30) + : : : +- CometFilter (29) + : : : +- CometScan parquet spark_catalog.default.catalog_returns (28) + : : +- ReusedExchange (32) + : +- CometBroadcastExchange (37) + : +- CometFilter (36) + : +- CometScan parquet spark_catalog.default.catalog_page (35) + +- * HashAggregate (70) + +- * ColumnarToRow (69) + +- CometColumnarExchange (68) + +- RowToColumnar (67) + +- * HashAggregate (66) + +- * ColumnarToRow (65) + +- CometProject (64) + +- CometBroadcastHashJoin (63) + :- CometProject (59) + : +- CometBroadcastHashJoin (58) + : :- CometUnion (56) + : : :- CometProject (48) + : : : +- CometFilter (47) + : : : +- CometScan parquet spark_catalog.default.web_sales (46) + : : +- CometProject (55) + : : +- CometBroadcastHashJoin (54) + : : :- CometBroadcastExchange (50) + : : : +- CometScan parquet spark_catalog.default.web_returns (49) + : : +- CometProject (53) + : : +- CometFilter (52) + : : +- CometScan parquet spark_catalog.default.web_sales (51) + : +- ReusedExchange (57) + +- CometBroadcastExchange (62) + +- CometFilter (61) + +- CometScan parquet spark_catalog.default.web_site (60) (1) Scan parquet spark_catalog.default.store_sales @@ -169,18 +177,24 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledV 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] -(21) Exchange +(21) RowToColumnar 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] -(22) HashAggregate [codegen id : 2] +(22) CometColumnarExchange +Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] +Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(23) ColumnarToRow [codegen id : 2] +Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] + +(24) 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))#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] -(23) Scan parquet spark_catalog.default.catalog_sales +(25) 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 [] @@ -188,15 +202,15 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#46), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct -(24) CometFilter +(26) 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) -(25) CometProject +(27) 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] -(26) Scan parquet spark_catalog.default.catalog_returns +(28) 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 [] @@ -204,76 +218,82 @@ PartitionFilters: [isnotnull(cr_returned_date_sk#57), dynamicpruningexpression(c PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct -(27) CometFilter +(29) 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) -(28) CometProject +(30) 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] -(29) CometUnion +(31) 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] -(30) ReusedExchange [Reuses operator id: 11] +(32) ReusedExchange [Reuses operator id: 11] Output [1]: [d_date_sk#64] -(31) CometBroadcastHashJoin +(33) 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 -(32) CometProject +(34) 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] -(33) Scan parquet spark_catalog.default.catalog_page +(35) 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 -(34) CometFilter +(36) CometFilter Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] Condition : isnotnull(cp_catalog_page_sk#65) -(35) CometBroadcastExchange +(37) 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 +(38) 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 +(39) 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] +(40) 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] +(41) 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 +(42) RowToColumnar +Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] + +(43) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] + +(44) ColumnarToRow [codegen id : 4] 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] +(45) 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 +(46) 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 [] @@ -281,178 +301,190 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#87), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(43) CometFilter +(47) 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) -(44) CometProject +(48) 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] -(45) Scan parquet spark_catalog.default.web_returns +(49) 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#99), dynamicpruningexpression(wr_returned_date_sk#99 IN dynamicpruning#88)] ReadSchema: struct -(46) CometBroadcastExchange +(50) 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] -(47) Scan parquet spark_catalog.default.web_sales +(51) 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 -(48) CometFilter +(52) 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) CometProject +(53) 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) CometBroadcastHashJoin +(54) 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) CometProject +(55) 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) CometUnion +(56) 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: 11] +(57) ReusedExchange [Reuses operator id: 11] Output [1]: [d_date_sk#110] -(54) CometBroadcastHashJoin +(58) 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) CometProject +(59) 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 +(60) Scan parquet spark_catalog.default.web_site 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 +(61) CometFilter Input [2]: [web_site_sk#111, web_site_id#112] Condition : isnotnull(web_site_sk#111) -(58) CometBroadcastExchange +(62) CometBroadcastExchange Input [2]: [web_site_sk#111, web_site_id#112] Arguments: [web_site_sk#111, web_site_id#112] -(59) CometBroadcastHashJoin +(63) 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) CometProject +(64) 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) ColumnarToRow [codegen id : 5] +(65) 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 : 5] +(66) 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 +(67) RowToColumnar +Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] + +(68) CometColumnarExchange +Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] +Arguments: hashpartitioning(web_site_id#112, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(69) ColumnarToRow [codegen 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 : 6] +(70) 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 +(71) Union -(66) Expand [codegen id : 7] +(72) 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 : 7] +(73) 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 +(74) RowToColumnar +Input [9]: [channel#130, id#131, spark_grouping_id#132, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] + +(75) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=4] + +(76) ColumnarToRow [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] -Arguments: hashpartitioning(channel#130, id#131, spark_grouping_id#132, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(69) HashAggregate [codegen id : 8] +(77) 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 +(78) TakeOrderedAndProject 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 ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (75) -+- * ColumnarToRow (74) - +- CometProject (73) - +- CometFilter (72) - +- CometScan parquet spark_catalog.default.date_dim (71) +BroadcastExchange (83) ++- * ColumnarToRow (82) + +- CometProject (81) + +- CometFilter (80) + +- CometScan parquet spark_catalog.default.date_dim (79) -(71) Scan parquet spark_catalog.default.date_dim +(79) 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 -(72) CometFilter +(80) 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)) -(73) CometProject +(81) CometProject Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(74) ColumnarToRow [codegen id : 1] +(82) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(75) BroadcastExchange +(83) BroadcastExchange Input [1]: [d_date_sk#22] 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 = 23 Hosting Expression = cs_sold_date_sk#46 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 25 Hosting Expression = cs_sold_date_sk#46 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#57 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 28 Hosting Expression = cr_returned_date_sk#57 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#87 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#87 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 45 Hosting Expression = wr_returned_date_sk#99 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 49 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 ceb3e9213..9e3aaf3d7 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,99 +1,107 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] 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 (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 (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 (1) - HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - ColumnarToRow - InputAdapter - CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] - CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - 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,ss_ext_sales_price,ss_net_profit,ss_sold_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_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 - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometFilter [s_store_sk,s_store_id] - 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] #6 - WholeStageCodegen (3) - HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - ColumnarToRow - InputAdapter - CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - 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,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_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 - ReusedExchange [d_date_sk] #4 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #7 - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - 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] #8 - WholeStageCodegen (5) - HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - ColumnarToRow - InputAdapter - CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] - CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - 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,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - 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,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] - CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #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_web_site_sk,ws_order_number,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - ReusedExchange [d_date_sk] #4 - CometBroadcastExchange [web_site_sk,web_site_id] #10 - CometFilter [web_site_sk,web_site_id] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel,id,spark_grouping_id] #1 + RowToColumnar + 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 (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] + ColumnarToRow + InputAdapter + CometColumnarExchange [s_store_id] #2 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + ColumnarToRow + InputAdapter + CometProject [sales_price,profit,return_amt,net_loss,s_store_id] + CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] + CometProject [store_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + 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,ss_ext_sales_price,ss_net_profit,ss_sold_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_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 + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [cp_catalog_page_id] #6 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + ColumnarToRow + InputAdapter + CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [page_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + 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,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_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 + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #7 + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [web_site_id] #8 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + ColumnarToRow + InputAdapter + CometProject [sales_price,profit,return_amt,net_loss,web_site_id] + CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] + CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + 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,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + 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,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] + CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #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_web_site_sk,ws_order_number,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange [web_site_sk,web_site_id] #10 + CometFilter [web_site_sk,web_site_id] + 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 ff35888ee..856d6238b 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 @@ -1,32 +1,34 @@ == Physical Plan == -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.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) +TakeOrderedAndProject (30) ++- * HashAggregate (29) + +- * ColumnarToRow (28) + +- CometColumnarExchange (27) + +- RowToColumnar (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.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 @@ -152,50 +154,56 @@ Functions [5]: [partial_sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk# 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 +(26) RowToColumnar 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 : 2] +(27) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(28) ColumnarToRow [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] + +(29) 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)#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 +(30) 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 #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 = 3 Hosting Expression = sr_returned_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (33) -+- * ColumnarToRow (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan parquet spark_catalog.default.date_dim (29) +BroadcastExchange (35) ++- * ColumnarToRow (34) + +- CometProject (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.date_dim (31) -(29) Scan parquet spark_catalog.default.date_dim +(31) 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 -(30) CometFilter +(32) 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)) -(31) CometProject +(33) CometProject 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] +(34) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#23] -(33) BroadcastExchange +(35) BroadcastExchange Input [1]: [d_date_sk#23] 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 aa3f7f2cf..ae77bbd8a 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,40 +1,42 @@ 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 (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 (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] - 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 [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,d_date_sk] + ColumnarToRow + InputAdapter + CometColumnarExchange [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 + RowToColumnar + 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] + 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 [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,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,ss_sold_date_sk,sr_returned_date_sk,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] - CometProject [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] #2 - CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [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] #4 - CometFilter [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] - 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 [d_date_sk] #5 - CometFilter [d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastHashJoin [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,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,ss_sold_date_sk,sr_returned_date_sk,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] + CometProject [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] #2 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_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_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [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] #4 + CometFilter [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] + 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 [d_date_sk] #5 + CometFilter [d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + 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 20fdfd4a6..bb3968041 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,44 +1,55 @@ == Physical Plan == -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) +* ColumnarToRow (51) ++- CometTakeOrderedAndProject (50) + +- CometFilter (49) + +- CometWindowExec (48) + +- CometSort (47) + +- CometColumnarExchange (46) + +- CometProject (45) + +- CometSortMergeJoin (44) + :- CometSort (23) + : +- CometColumnarExchange (22) + : +- RowToColumnar (21) + : +- * Project (20) + : +- Window (19) + : +- * ColumnarToRow (18) + : +- CometSort (17) + : +- CometColumnarExchange (16) + : +- RowToColumnar (15) + : +- * HashAggregate (14) + : +- * ColumnarToRow (13) + : +- CometColumnarExchange (12) + : +- RowToColumnar (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) + +- CometSort (43) + +- CometColumnarExchange (42) + +- RowToColumnar (41) + +- * Project (40) + +- Window (39) + +- * ColumnarToRow (38) + +- CometSort (37) + +- CometColumnarExchange (36) + +- RowToColumnar (35) + +- * HashAggregate (34) + +- * ColumnarToRow (33) + +- CometColumnarExchange (32) + +- RowToColumnar (31) + +- * HashAggregate (30) + +- * ColumnarToRow (29) + +- CometProject (28) + +- CometBroadcastHashJoin (27) + :- CometFilter (25) + : +- CometScan parquet spark_catalog.default.store_sales (24) + +- ReusedExchange (26) (1) Scan parquet spark_catalog.default.web_sales @@ -91,42 +102,57 @@ Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] Aggregate Attributes [1]: [sum#8] Results [3]: [ws_item_sk#1, d_date#6, sum#9] -(11) Exchange +(11) RowToColumnar 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] -(12) HashAggregate [codegen id : 2] +(12) CometColumnarExchange +Input [3]: [ws_item_sk#1, d_date#6, sum#9] +Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(13) ColumnarToRow [codegen id : 2] +Input [3]: [ws_item_sk#1, d_date#6, sum#9] + +(14) 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))#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] -(13) Exchange +(15) RowToColumnar +Input [4]: [item_sk#11, d_date#6, _w0#12, ws_item_sk#1] + +(16) CometColumnarExchange 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] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(14) Sort [codegen id : 3] +(17) CometSort 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 +Arguments: [item_sk#11, d_date#6, _w0#12, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(15) Window +(18) ColumnarToRow [codegen id : 3] +Input [4]: [item_sk#11, d_date#6, _w0#12, ws_item_sk#1] + +(19) 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] -(16) Project [codegen id : 4] +(20) 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] -(17) Exchange +(21) RowToColumnar 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] -(18) Sort [codegen id : 5] +(22) CometColumnarExchange 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 +Arguments: hashpartitioning(item_sk#11, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(19) Scan parquet spark_catalog.default.store_sales +(23) CometSort +Input [3]: [item_sk#11, d_date#6, cume_sales#13] +Arguments: [item_sk#11, d_date#6, cume_sales#13], [item_sk#11 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] + +(24) 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 [] @@ -134,129 +160,146 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#16), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(20) CometFilter +(25) 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] +(26) ReusedExchange [Reuses operator id: 6] Output [2]: [d_date_sk#18, d_date#19] -(22) CometBroadcastHashJoin +(27) 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 +(28) 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] +(29) ColumnarToRow [codegen id : 5] Input [3]: [ss_item_sk#14, ss_sales_price#15, d_date#19] -(25) HashAggregate [codegen id : 6] +(30) HashAggregate [codegen id : 5] 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 +(31) RowToColumnar +Input [3]: [ss_item_sk#14, d_date#19, sum#21] + +(32) CometColumnarExchange +Input [3]: [ss_item_sk#14, d_date#19, sum#21] +Arguments: hashpartitioning(ss_item_sk#14, d_date#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(33) ColumnarToRow [codegen id : 6] 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] +(34) HashAggregate [codegen id : 6] 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 +(35) RowToColumnar 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] +(36) CometColumnarExchange 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 +Arguments: hashpartitioning(ss_item_sk#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(30) Window +(37) CometSort +Input [4]: [item_sk#23, d_date#19, _w0#24, ss_item_sk#14] +Arguments: [item_sk#23, d_date#19, _w0#24, ss_item_sk#14], [ss_item_sk#14 ASC NULLS FIRST, d_date#19 ASC NULLS FIRST] + +(38) ColumnarToRow [codegen id : 7] +Input [4]: [item_sk#23, d_date#19, _w0#24, ss_item_sk#14] + +(39) 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] +(40) Project [codegen id : 8] 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 +(41) RowToColumnar +Input [3]: [item_sk#23, d_date#19, cume_sales#25] + +(42) CometColumnarExchange 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] +Arguments: hashpartitioning(item_sk#23, d_date#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(33) Sort [codegen id : 10] +(43) CometSort 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 +Arguments: [item_sk#23, d_date#19, cume_sales#25], [item_sk#23 ASC NULLS FIRST, d_date#19 ASC NULLS FIRST] -(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 +(44) CometSortMergeJoin +Left output [3]: [item_sk#11, d_date#6, cume_sales#13] +Right output [3]: [item_sk#23, d_date#19, cume_sales#25] +Arguments: [item_sk#11, d_date#6], [item_sk#23, d_date#19], FullOuter -(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] +(45) CometProject Input [6]: [item_sk#11, d_date#6, cume_sales#13, item_sk#23, d_date#19, cume_sales#25] +Arguments: [item_sk#26, d_date#27, web_sales#28, store_sales#29], [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] -(36) Exchange +(46) CometColumnarExchange 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] +Arguments: hashpartitioning(item_sk#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(37) Sort [codegen id : 12] +(47) CometSort 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 +Arguments: [item_sk#26, d_date#27, web_sales#28, store_sales#29], [item_sk#26 ASC NULLS FIRST, d_date#27 ASC NULLS FIRST] -(38) Window +(48) CometWindowExec 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] +Arguments: [item_sk#26, d_date#27, web_sales#28, store_sales#29, web_cumulative#30, store_cumulative#31], [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] -(39) Filter [codegen id : 13] +(49) CometFilter 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)) -(40) TakeOrderedAndProject +(50) CometTakeOrderedAndProject +Input [6]: [item_sk#26, d_date#27, web_sales#28, store_sales#29, web_cumulative#30, store_cumulative#31] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_sk#26 ASC NULLS FIRST,d_date#27 ASC NULLS FIRST], output=[item_sk#26,d_date#27,web_sales#28,store_sales#29,web_cumulative#30,store_cumulative#31]), [item_sk#26, d_date#27, web_sales#28, store_sales#29, web_cumulative#30, store_cumulative#31], 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] + +(51) ColumnarToRow [codegen id : 9] 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 (45) -+- * ColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan parquet spark_catalog.default.date_dim (41) +BroadcastExchange (56) ++- * ColumnarToRow (55) + +- CometProject (54) + +- CometFilter (53) + +- CometScan parquet spark_catalog.default.date_dim (52) -(41) Scan parquet spark_catalog.default.date_dim +(52) 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 -(42) CometFilter +(53) 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)) -(43) CometProject +(54) 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] -(44) ColumnarToRow [codegen id : 1] +(55) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(45) BroadcastExchange +(56) 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 = 19 Hosting Expression = ss_sold_date_sk#16 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 24 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 f19312ca8..6738c7cef 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,32 +1,32 @@ -TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (13) - Filter [web_cumulative,store_cumulative] - InputAdapter - Window [web_sales,item_sk,d_date,store_sales] - WholeStageCodegen (12) - Sort [item_sk,d_date] - InputAdapter - Exchange [item_sk] #1 - 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 (5) - Sort [item_sk,d_date] +WholeStageCodegen (9) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + CometFilter [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + CometWindowExec [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + CometSort [item_sk,d_date,web_sales,store_sales] + CometColumnarExchange [item_sk] #1 + CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] + CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #2 + RowToColumnar + WholeStageCodegen (4) + Project [item_sk,d_date,cume_sales] InputAdapter - Exchange [item_sk,d_date] #2 - WholeStageCodegen (4) - Project [item_sk,d_date,cume_sales] + Window [_w0,ws_item_sk,d_date] + WholeStageCodegen (3) + ColumnarToRow InputAdapter - Window [_w0,ws_item_sk,d_date] - WholeStageCodegen (3) - Sort [ws_item_sk,d_date] - InputAdapter - Exchange [ws_item_sk] #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 + CometSort [item_sk,d_date,_w0,ws_item_sk] + CometColumnarExchange [ws_item_sk] #3 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,_w0,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [ws_item_sk,d_date] #4 + RowToColumnar WholeStageCodegen (1) HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] ColumnarToRow @@ -47,24 +47,26 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - WholeStageCodegen (10) - Sort [item_sk,d_date] + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #7 + RowToColumnar + WholeStageCodegen (8) + Project [item_sk,d_date,cume_sales] InputAdapter - Exchange [item_sk,d_date] #7 - WholeStageCodegen (9) - Project [item_sk,d_date,cume_sales] + Window [_w0,ss_item_sk,d_date] + WholeStageCodegen (7) + ColumnarToRow InputAdapter - Window [_w0,ss_item_sk,d_date] - WholeStageCodegen (8) - Sort [ss_item_sk,d_date] - InputAdapter - 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] #9 - WholeStageCodegen (6) + CometSort [item_sk,d_date,_w0,ss_item_sk] + CometColumnarExchange [ss_item_sk] #8 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,_w0,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [ss_item_sk,d_date] #9 + RowToColumnar + WholeStageCodegen (5) HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] ColumnarToRow InputAdapter 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 6d4feea57..3feea9a63 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,23 +1,25 @@ == Physical Plan == -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) +TakeOrderedAndProject (21) ++- * HashAggregate (20) + +- * ColumnarToRow (19) + +- CometColumnarExchange (18) + +- RowToColumnar (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 @@ -98,18 +100,24 @@ 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] -(17) Exchange +(17) RowToColumnar 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=1] -(18) HashAggregate [codegen id : 2] +(18) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(19) ColumnarToRow [codegen id : 2] +Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] + +(20) 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] -(19) TakeOrderedAndProject +(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] 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 d7661b7be..9acfebf22 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,23 +1,25 @@ TakeOrderedAndProject [d_year,ext_price,brand_id,brand] 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 (1) - HashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] - CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] + ColumnarToRow + InputAdapter + CometColumnarExchange [d_year,i_brand,i_brand_id] #1 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] + CometProject [d_year,ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometProject [d_date_sk,d_year] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 + CometProject [i_item_sk,i_brand_id,i_brand] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] + 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 e85d73a07..5ed76db88 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,33 +1,37 @@ == Physical Plan == -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) +TakeOrderedAndProject (33) ++- * Project (32) + +- * Filter (31) + +- Window (30) + +- * ColumnarToRow (29) + +- CometSort (28) + +- CometColumnarExchange (27) + +- RowToColumnar (26) + +- * HashAggregate (25) + +- * ColumnarToRow (24) + +- CometColumnarExchange (23) + +- RowToColumnar (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 @@ -132,70 +136,82 @@ Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] Aggregate Attributes [1]: [sum#19] Results [3]: [i_manufact_id#5, d_qoy#17, sum#20] -(22) Exchange +(22) RowToColumnar 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] -(23) HashAggregate [codegen id : 2] +(23) CometColumnarExchange +Input [3]: [i_manufact_id#5, d_qoy#17, sum#20] +Arguments: hashpartitioning(i_manufact_id#5, d_qoy#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(24) ColumnarToRow [codegen id : 2] +Input [3]: [i_manufact_id#5, d_qoy#17, sum#20] + +(25) 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))#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] -(24) Exchange +(26) RowToColumnar +Input [3]: [i_manufact_id#5, sum_sales#22, _w0#23] + +(27) CometColumnarExchange +Input [3]: [i_manufact_id#5, sum_sales#22, _w0#23] +Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(28) CometSort Input [3]: [i_manufact_id#5, sum_sales#22, _w0#23] -Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: [i_manufact_id#5, sum_sales#22, _w0#23], [i_manufact_id#5 ASC NULLS FIRST] -(25) Sort [codegen id : 3] +(29) ColumnarToRow [codegen id : 3] Input [3]: [i_manufact_id#5, sum_sales#22, _w0#23] -Arguments: [i_manufact_id#5 ASC NULLS FIRST], false, 0 -(26) Window +(30) 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] -(27) Filter [codegen id : 4] +(31) 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 -(28) Project [codegen id : 4] +(32) 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] -(29) TakeOrderedAndProject +(33) 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 = 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) +BroadcastExchange (38) ++- * ColumnarToRow (37) + +- CometProject (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.date_dim (34) -(30) Scan parquet spark_catalog.default.date_dim +(34) 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 -(31) CometFilter +(35) 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)) -(32) CometProject +(36) 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] -(33) ColumnarToRow [codegen id : 1] +(37) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#15, d_qoy#17] -(34) BroadcastExchange +(38) 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 b90cb42d0..f08e3a806 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 @@ -5,41 +5,45 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] InputAdapter Window [_w0,i_manufact_id] WholeStageCodegen (3) - Sort [i_manufact_id] + ColumnarToRow InputAdapter - Exchange [i_manufact_id] #1 - 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 (1) - HashAggregate [i_manufact_id,d_qoy,ss_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [i_manufact_id,ss_sales_price,d_qoy] - CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] - CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] - CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] - CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_item_sk,i_manufact_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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_date_sk,d_month_seq,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [d_date_sk,d_qoy] #5 - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] + CometSort [i_manufact_id,sum_sales,_w0] + CometColumnarExchange [i_manufact_id] #1 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [i_manufact_id,d_qoy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_manufact_id,d_qoy] #2 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [i_manufact_id,d_qoy,ss_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [i_manufact_id,ss_sales_price,d_qoy] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] + CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] + CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_item_sk,i_manufact_id] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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_date_sk,d_month_seq,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometBroadcastExchange [d_date_sk,d_qoy] #5 + CometProject [d_date_sk,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometBroadcastExchange [s_store_sk] #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 953223fd9..050301d07 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,60 +1,64 @@ == Physical Plan == -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) +TakeOrderedAndProject (60) ++- * HashAggregate (59) + +- * ColumnarToRow (58) + +- CometColumnarExchange (57) + +- RowToColumnar (56) + +- * HashAggregate (55) + +- * HashAggregate (54) + +- * ColumnarToRow (53) + +- CometColumnarExchange (52) + +- RowToColumnar (51) + +- * HashAggregate (50) + +- * ColumnarToRow (49) + +- CometProject (48) + +- CometBroadcastHashJoin (47) + :- CometProject (42) + : +- CometBroadcastHashJoin (41) + : :- CometProject (37) + : : +- CometBroadcastHashJoin (36) + : : :- CometProject (32) + : : : +- CometBroadcastHashJoin (31) + : : : :- CometHashAggregate (27) + : : : : +- CometColumnarExchange (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) + : : : +- CometBroadcastExchange (30) + : : : +- CometFilter (29) + : : : +- CometScan parquet spark_catalog.default.store_sales (28) + : : +- CometBroadcastExchange (35) + : : +- CometFilter (34) + : : +- CometScan parquet spark_catalog.default.customer_address (33) + : +- CometBroadcastExchange (40) + : +- CometFilter (39) + : +- CometScan parquet spark_catalog.default.store (38) + +- CometBroadcastExchange (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) (1) Scan parquet spark_catalog.default.catalog_sales @@ -178,21 +182,16 @@ Input [2]: [c_customer_sk#20, c_current_addr_sk#21] Keys [2]: [c_customer_sk#20, c_current_addr_sk#21] Functions: [] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(27) 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#20, c_current_addr_sk#21] -(29) Scan parquet spark_catalog.default.store_sales +(28) 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 [] @@ -200,290 +199,312 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#24), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(30) CometFilter +(29) CometFilter Input [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] Condition : isnotnull(ss_customer_sk#22) -(31) ColumnarToRow [codegen id : 2] +(30) CometBroadcastExchange Input [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] +Arguments: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -(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] +(31) CometBroadcastHashJoin +Left output [2]: [c_customer_sk#20, c_current_addr_sk#21] +Right output [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] +Arguments: [c_customer_sk#20], [ss_customer_sk#22], Inner, BuildRight -(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 - -(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] +(32) CometProject 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] +Arguments: [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24], [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24] -(35) Scan parquet spark_catalog.default.customer_address +(33) 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 -(36) CometFilter +(34) 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)) -(37) ColumnarToRow [codegen id : 3] +(35) CometBroadcastExchange Input [3]: [ca_address_sk#26, ca_county#27, ca_state#28] +Arguments: [ca_address_sk#26, ca_county#27, ca_state#28] -(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] +(36) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24] +Right output [3]: [ca_address_sk#26, ca_county#27, ca_state#28] +Arguments: [c_current_addr_sk#21], [ca_address_sk#26], Inner, BuildRight -(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 - -(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] +(37) CometProject 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] +Arguments: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#28], [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#28] -(41) Scan parquet spark_catalog.default.store +(38) 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 -(42) CometFilter +(39) CometFilter Input [2]: [s_county#29, s_state#30] Condition : (isnotnull(s_county#29) AND isnotnull(s_state#30)) -(43) ColumnarToRow [codegen id : 4] -Input [2]: [s_county#29, s_state#30] - -(44) BroadcastExchange +(40) CometBroadcastExchange Input [2]: [s_county#29, s_state#30] -Arguments: HashedRelationBroadcastMode(List(input[0, string, false], input[1, string, false]),false), [plan_id=4] +Arguments: [s_county#29, s_state#30] -(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 +(41) CometBroadcastHashJoin +Left output [5]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#28] +Right output [2]: [s_county#29, s_state#30] +Arguments: [ca_county#27, ca_state#28], [s_county#29, s_state#30], Inner, BuildRight -(46) Project [codegen id : 6] -Output [3]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24] +(42) CometProject 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] +Arguments: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24], [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24] -(47) ReusedExchange [Reuses operator id: 66] -Output [1]: [d_date_sk#31] +(43) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#31, d_month_seq#32] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,ScalarSubquery#33), LessThanOrEqual(d_month_seq,ScalarSubquery#34), IsNotNull(d_date_sk)] +ReadSchema: struct + +(44) CometFilter +Input [2]: [d_date_sk#31, d_month_seq#32] +Condition : (((isnotnull(d_month_seq#32) AND (d_month_seq#32 >= ReusedSubquery Subquery scalar-subquery#33, [id=#35])) AND (d_month_seq#32 <= ReusedSubquery Subquery scalar-subquery#34, [id=#36])) AND isnotnull(d_date_sk#31)) + +(45) CometProject +Input [2]: [d_date_sk#31, d_month_seq#32] +Arguments: [d_date_sk#31], [d_date_sk#31] + +(46) CometBroadcastExchange +Input [1]: [d_date_sk#31] +Arguments: [d_date_sk#31] -(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 +(47) CometBroadcastHashJoin +Left output [3]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24] +Right output [1]: [d_date_sk#31] +Arguments: [ss_sold_date_sk#24], [d_date_sk#31], Inner, BuildRight -(49) Project [codegen id : 6] -Output [2]: [c_customer_sk#20, ss_ext_sales_price#23] +(48) CometProject Input [4]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, d_date_sk#31] +Arguments: [c_customer_sk#20, ss_ext_sales_price#23], [c_customer_sk#20, ss_ext_sales_price#23] + +(49) ColumnarToRow [codegen id : 1] +Input [2]: [c_customer_sk#20, ss_ext_sales_price#23] -(50) HashAggregate [codegen id : 6] +(50) HashAggregate [codegen id : 1] 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] +Aggregate Attributes [1]: [sum#37] +Results [2]: [c_customer_sk#20, sum#38] + +(51) RowToColumnar +Input [2]: [c_customer_sk#20, sum#38] + +(52) CometColumnarExchange +Input [2]: [c_customer_sk#20, sum#38] +Arguments: hashpartitioning(c_customer_sk#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(51) Exchange -Input [2]: [c_customer_sk#20, sum#33] -Arguments: hashpartitioning(c_customer_sk#20, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(53) ColumnarToRow [codegen id : 2] +Input [2]: [c_customer_sk#20, sum#38] -(52) HashAggregate [codegen id : 7] -Input [2]: [c_customer_sk#20, sum#33] +(54) HashAggregate [codegen id : 2] +Input [2]: [c_customer_sk#20, sum#38] 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] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#23))#39] +Results [1]: [cast((MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#23))#39,17,2) / 50) as int) AS segment#40] -(53) HashAggregate [codegen id : 7] -Input [1]: [segment#35] -Keys [1]: [segment#35] +(55) HashAggregate [codegen id : 2] +Input [1]: [segment#40] +Keys [1]: [segment#40] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#36] -Results [2]: [segment#35, count#37] +Aggregate Attributes [1]: [count#41] +Results [2]: [segment#40, count#42] + +(56) RowToColumnar +Input [2]: [segment#40, count#42] + +(57) CometColumnarExchange +Input [2]: [segment#40, count#42] +Arguments: hashpartitioning(segment#40, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(54) Exchange -Input [2]: [segment#35, count#37] -Arguments: hashpartitioning(segment#35, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(58) ColumnarToRow [codegen id : 3] +Input [2]: [segment#40, count#42] -(55) HashAggregate [codegen id : 8] -Input [2]: [segment#35, count#37] -Keys [1]: [segment#35] +(59) HashAggregate [codegen id : 3] +Input [2]: [segment#40, count#42] +Keys [1]: [segment#40] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#38] -Results [3]: [segment#35, count(1)#38 AS num_customers#39, (segment#35 * 50) AS segment_base#40] +Aggregate Attributes [1]: [count(1)#43] +Results [3]: [segment#40, count(1)#43 AS num_customers#44, (segment#40 * 50) AS segment_base#45] -(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] +(60) TakeOrderedAndProject +Input [3]: [segment#40, num_customers#44, segment_base#45] +Arguments: 100, [segment#40 ASC NULLS FIRST, num_customers#44 ASC NULLS FIRST], [segment#40, num_customers#44, segment_base#45] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (61) -+- * ColumnarToRow (60) - +- CometProject (59) - +- CometFilter (58) - +- CometScan parquet spark_catalog.default.date_dim (57) +BroadcastExchange (65) ++- * ColumnarToRow (64) + +- CometProject (63) + +- CometFilter (62) + +- CometScan parquet spark_catalog.default.date_dim (61) -(57) Scan parquet spark_catalog.default.date_dim +(61) 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 -(58) CometFilter +(62) 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)) -(59) CometProject +(63) CometProject Input [3]: [d_date_sk#17, d_year#18, d_moy#19] Arguments: [d_date_sk#17], [d_date_sk#17] -(60) ColumnarToRow [codegen id : 1] +(64) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#17] -(61) BroadcastExchange +(65) BroadcastExchange Input [1]: [d_date_sk#17] -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=4] Subquery:2 Hosting operator id = 4 Hosting Expression = ws_sold_date_sk#10 IN dynamicpruning#4 -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) +Subquery:3 Hosting operator id = 28 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#25 +BroadcastExchange (70) ++- * ColumnarToRow (69) + +- CometProject (68) + +- CometFilter (67) + +- CometScan parquet spark_catalog.default.date_dim (66) -(62) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#31, d_month_seq#41] +(66) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#31, d_month_seq#32] 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)] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,ScalarSubquery#33), LessThanOrEqual(d_month_seq,ScalarSubquery#34), IsNotNull(d_date_sk)] ReadSchema: struct -(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)) +(67) CometFilter +Input [2]: [d_date_sk#31, d_month_seq#32] +Condition : (((isnotnull(d_month_seq#32) AND (d_month_seq#32 >= ReusedSubquery Subquery scalar-subquery#33, [id=#35])) AND (d_month_seq#32 <= ReusedSubquery Subquery scalar-subquery#34, [id=#36])) AND isnotnull(d_date_sk#31)) -(64) CometProject -Input [2]: [d_date_sk#31, d_month_seq#41] +(68) CometProject +Input [2]: [d_date_sk#31, d_month_seq#32] Arguments: [d_date_sk#31], [d_date_sk#31] -(65) ColumnarToRow [codegen id : 1] +(69) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#31] -(66) BroadcastExchange +(70) BroadcastExchange Input [1]: [d_date_sk#31] -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:4 Hosting operator id = 63 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#44] +Subquery:4 Hosting operator id = 67 Hosting Expression = ReusedSubquery Subquery scalar-subquery#33, [id=#35] -Subquery:5 Hosting operator id = 63 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#45] +Subquery:5 Hosting operator id = 67 Hosting Expression = ReusedSubquery Subquery scalar-subquery#34, [id=#36] -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) +Subquery:6 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#33, [id=#35] +* ColumnarToRow (77) ++- CometHashAggregate (76) + +- CometColumnarExchange (75) + +- CometHashAggregate (74) + +- CometProject (73) + +- CometFilter (72) + +- CometScan parquet spark_catalog.default.date_dim (71) -(67) Scan parquet spark_catalog.default.date_dim +(71) 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 -(68) CometFilter +(72) 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)) -(69) CometProject +(73) 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] -(70) CometHashAggregate +(74) CometHashAggregate Input [1]: [(d_month_seq + 1)#49] Keys [1]: [(d_month_seq + 1)#49] Functions: [] -(71) ColumnarToRow [codegen id : 1] +(75) CometColumnarExchange Input [1]: [(d_month_seq + 1)#49] +Arguments: hashpartitioning((d_month_seq + 1)#49, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(72) Exchange -Input [1]: [(d_month_seq + 1)#49] -Arguments: hashpartitioning((d_month_seq + 1)#49, 5), ENSURE_REQUIREMENTS, [plan_id=9] - -(73) HashAggregate [codegen id : 2] +(76) CometHashAggregate 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 = 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) +(77) ColumnarToRow [codegen id : 1] +Input [1]: [(d_month_seq + 1)#49] + +Subquery:7 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#34, [id=#36] +* ColumnarToRow (84) ++- CometHashAggregate (83) + +- CometColumnarExchange (82) + +- CometHashAggregate (81) + +- CometProject (80) + +- CometFilter (79) + +- CometScan parquet spark_catalog.default.date_dim (78) -(74) Scan parquet spark_catalog.default.date_dim +(78) 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 -(75) CometFilter +(79) 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)) -(76) CometProject +(80) 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] -(77) CometHashAggregate +(81) CometHashAggregate Input [1]: [(d_month_seq + 3)#53] Keys [1]: [(d_month_seq + 3)#53] Functions: [] -(78) ColumnarToRow [codegen id : 1] +(82) CometColumnarExchange Input [1]: [(d_month_seq + 3)#53] +Arguments: hashpartitioning((d_month_seq + 3)#53, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(79) Exchange -Input [1]: [(d_month_seq + 3)#53] -Arguments: hashpartitioning((d_month_seq + 3)#53, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(80) HashAggregate [codegen id : 2] +(83) CometHashAggregate Input [1]: [(d_month_seq + 3)#53] Keys [1]: [(d_month_seq + 3)#53] Functions: [] -Aggregate Attributes: [] -Results [1]: [(d_month_seq + 3)#53] + +(84) ColumnarToRow [codegen id : 1] +Input [1]: [(d_month_seq + 3)#53] + +Subquery:8 Hosting operator id = 44 Hosting Expression = ReusedSubquery Subquery scalar-subquery#33, [id=#35] + +Subquery:9 Hosting operator id = 44 Hosting Expression = ReusedSubquery Subquery scalar-subquery#34, [id=#36] + +Subquery:10 Hosting operator id = 43 Hosting Expression = ReusedSubquery Subquery scalar-subquery#33, [id=#35] + +Subquery:11 Hosting operator id = 43 Hosting Expression = ReusedSubquery Subquery scalar-subquery#34, [id=#36] 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 a8df48770..e1696b97d 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,117 +1,109 @@ TakeOrderedAndProject [segment,num_customers,segment_base] - WholeStageCodegen (8) + WholeStageCodegen (3) HashAggregate [segment,count] [count(1),num_customers,segment_base,count] - InputAdapter - Exchange [segment] #1 - 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 (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] - Project [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [ca_county,ca_state,s_county,s_state] - Project [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - HashAggregate [c_customer_sk,c_current_addr_sk] - InputAdapter - Exchange [c_customer_sk,c_current_addr_sk] #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometColumnarExchange [segment] #1 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [segment] [count,count] + HashAggregate [c_customer_sk,sum] [sum(UnscaledValue(ss_ext_sales_price)),segment,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_sk] #2 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [c_customer_sk,ss_ext_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [c_customer_sk,ss_ext_sales_price] + CometBroadcastHashJoin [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometProject [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state,s_county,s_state] + CometProject [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state] + CometBroadcastHashJoin [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk,ca_address_sk,ca_county,ca_state] + CometProject [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_addr_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] CometHashAggregate [c_customer_sk,c_current_addr_sk] - CometProject [c_customer_sk,c_current_addr_sk] - CometBroadcastHashJoin [customer_sk,c_customer_sk,c_current_addr_sk] - CometProject [customer_sk] - CometBroadcastHashJoin [sold_date_sk,customer_sk,d_date_sk] - CometProject [sold_date_sk,customer_sk] - CometBroadcastHashJoin [sold_date_sk,customer_sk,item_sk,i_item_sk] - CometUnion [sold_date_sk,customer_sk,item_sk] - CometProject [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [i_item_sk] #5 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_class,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #7 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk,ss_ext_sales_price,ss_sold_date_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 #9 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [(d_month_seq + 1)] #3 - ReusedSubquery [(d_month_seq + 3)] #4 - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - Subquery #3 - WholeStageCodegen (2) - HashAggregate [(d_month_seq + 1)] - InputAdapter - Exchange [(d_month_seq + 1)] #10 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + CometColumnarExchange [c_customer_sk,c_current_addr_sk] #3 + CometHashAggregate [c_customer_sk,c_current_addr_sk] + CometProject [c_customer_sk,c_current_addr_sk] + CometBroadcastHashJoin [customer_sk,c_customer_sk,c_current_addr_sk] + CometProject [customer_sk] + CometBroadcastHashJoin [sold_date_sk,customer_sk,d_date_sk] + CometProject [sold_date_sk,customer_sk] + CometBroadcastHashJoin [sold_date_sk,customer_sk,item_sk,i_item_sk] + CometUnion [sold_date_sk,customer_sk,item_sk] + CometProject [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] [sold_date_sk,customer_sk,item_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [i_item_sk] #5 + CometProject [i_item_sk] + CometFilter [i_item_sk,i_class,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #7 + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] #8 + CometFilter [ss_customer_sk,ss_ext_sales_price,ss_sold_date_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 #9 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + ReusedSubquery [(d_month_seq + 1)] #3 + ReusedSubquery [(d_month_seq + 3)] #4 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + Subquery #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [(d_month_seq + 1)] + CometColumnarExchange [(d_month_seq + 1)] #10 CometHashAggregate [(d_month_seq + 1)] CometProject [d_month_seq] [(d_month_seq + 1)] CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - Subquery #4 - WholeStageCodegen (2) - HashAggregate [(d_month_seq + 3)] - InputAdapter - Exchange [(d_month_seq + 3)] #11 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + Subquery #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [(d_month_seq + 3)] + CometColumnarExchange [(d_month_seq + 3)] #11 CometHashAggregate [(d_month_seq + 3)] CometProject [d_month_seq] [(d_month_seq + 3)] CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - InputAdapter - 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 #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] #9 + CometBroadcastExchange [ca_address_sk,ca_county,ca_state] #12 + CometFilter [ca_address_sk,ca_county,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state] + CometBroadcastExchange [s_county,s_state] #13 + CometFilter [s_county,s_state] + CometScan parquet spark_catalog.default.store [s_county,s_state] + CometBroadcastExchange [d_date_sk] #14 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + ReusedSubquery [(d_month_seq + 1)] #3 + ReusedSubquery [(d_month_seq + 3)] #4 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + ReusedSubquery [(d_month_seq + 1)] #3 + ReusedSubquery [(d_month_seq + 3)] #4 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 dcd32a96a..36e8ac073 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,23 +1,25 @@ == Physical Plan == -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) +TakeOrderedAndProject (21) ++- * HashAggregate (20) + +- * ColumnarToRow (19) + +- CometColumnarExchange (18) + +- RowToColumnar (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 @@ -98,18 +100,24 @@ 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] -(17) Exchange +(17) RowToColumnar 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=1] -(18) HashAggregate [codegen id : 2] +(18) CometColumnarExchange +Input [3]: [i_brand#9, i_brand_id#8, sum#12] +Arguments: hashpartitioning(i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(19) ColumnarToRow [codegen id : 2] +Input [3]: [i_brand#9, i_brand_id#8, sum#12] + +(20) 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] -(19) TakeOrderedAndProject +(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] 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 2461ab830..290d5cb88 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,23 +1,25 @@ TakeOrderedAndProject [ext_price,brand_id,brand] 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 (1) - HashAggregate [i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] - CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_brand,i_brand_id] #1 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [ss_ext_sales_price,i_brand_id,i_brand] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 + CometProject [i_item_sk,i_brand_id,i_brand] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] + 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 bbfd9dcaf..b3e0794df 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 @@ -1,67 +1,75 @@ == Physical Plan == -TakeOrderedAndProject (63) -+- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- Union (59) - :- * HashAggregate (28) - : +- Exchange (27) - : +- * HashAggregate (26) - : +- * 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) - : +- * 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) - +- * 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) +TakeOrderedAndProject (71) ++- * HashAggregate (70) + +- * ColumnarToRow (69) + +- CometColumnarExchange (68) + +- RowToColumnar (67) + +- * HashAggregate (66) + +- Union (65) + :- * HashAggregate (30) + : +- * ColumnarToRow (29) + : +- CometColumnarExchange (28) + : +- RowToColumnar (27) + : +- * HashAggregate (26) + : +- * 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 (47) + : +- * ColumnarToRow (46) + : +- CometColumnarExchange (45) + : +- RowToColumnar (44) + : +- * HashAggregate (43) + : +- * ColumnarToRow (42) + : +- CometProject (41) + : +- CometBroadcastHashJoin (40) + : :- CometProject (38) + : : +- CometBroadcastHashJoin (37) + : : :- CometProject (35) + : : : +- CometBroadcastHashJoin (34) + : : : :- CometFilter (32) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (31) + : : : +- ReusedExchange (33) + : : +- ReusedExchange (36) + : +- ReusedExchange (39) + +- * HashAggregate (64) + +- * ColumnarToRow (63) + +- CometColumnarExchange (62) + +- RowToColumnar (61) + +- * HashAggregate (60) + +- * ColumnarToRow (59) + +- CometProject (58) + +- CometBroadcastHashJoin (57) + :- CometProject (55) + : +- CometBroadcastHashJoin (54) + : :- CometProject (52) + : : +- CometBroadcastHashJoin (51) + : : :- CometFilter (49) + : : : +- CometScan parquet spark_catalog.default.web_sales (48) + : : +- ReusedExchange (50) + : +- ReusedExchange (53) + +- ReusedExchange (56) (1) Scan parquet spark_catalog.default.store_sales @@ -190,18 +198,24 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] Aggregate Attributes [1]: [sum#15] Results [2]: [i_item_id#12, sum#16] -(27) Exchange +(27) RowToColumnar Input [2]: [i_item_id#12, sum#16] -Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(28) HashAggregate [codegen id : 2] +(28) CometColumnarExchange +Input [2]: [i_item_id#12, sum#16] +Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(29) ColumnarToRow [codegen id : 2] +Input [2]: [i_item_id#12, sum#16] + +(30) 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))#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 +(31) Scan parquet spark_catalog.default.catalog_sales 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 [] @@ -209,68 +223,74 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#22), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(30) CometFilter +(32) CometFilter 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] +(33) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#24] -(32) CometBroadcastHashJoin +(34) 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 +(35) 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] +(36) ReusedExchange [Reuses operator id: 12] Output [1]: [ca_address_sk#25] -(35) CometBroadcastHashJoin +(37) 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 +(38) 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] +(39) ReusedExchange [Reuses operator id: 22] Output [2]: [i_item_sk#26, i_item_id#27] -(38) CometBroadcastHashJoin +(40) 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 +(41) 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] +(42) ColumnarToRow [codegen id : 3] Input [2]: [cs_ext_sales_price#21, i_item_id#27] -(41) HashAggregate [codegen id : 3] +(43) 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 +(44) RowToColumnar +Input [2]: [i_item_id#27, sum#29] + +(45) CometColumnarExchange +Input [2]: [i_item_id#27, sum#29] +Arguments: hashpartitioning(i_item_id#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(46) ColumnarToRow [codegen id : 4] Input [2]: [i_item_id#27, sum#29] -Arguments: hashpartitioning(i_item_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(43) HashAggregate [codegen id : 4] +(47) 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 +(48) Scan parquet spark_catalog.default.web_sales 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 [] @@ -278,125 +298,137 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(45) CometFilter +(49) CometFilter 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] +(50) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#37] -(47) CometBroadcastHashJoin +(51) 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 +(52) 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] +(53) ReusedExchange [Reuses operator id: 12] Output [1]: [ca_address_sk#38] -(50) CometBroadcastHashJoin +(54) 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 +(55) 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] +(56) ReusedExchange [Reuses operator id: 22] Output [2]: [i_item_sk#39, i_item_id#40] -(53) CometBroadcastHashJoin +(57) 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 +(58) 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] +(59) ColumnarToRow [codegen id : 5] Input [2]: [ws_ext_sales_price#34, i_item_id#40] -(56) HashAggregate [codegen id : 5] +(60) 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 +(61) RowToColumnar Input [2]: [i_item_id#40, sum#42] -Arguments: hashpartitioning(i_item_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(58) HashAggregate [codegen id : 6] +(62) CometColumnarExchange +Input [2]: [i_item_id#40, sum#42] +Arguments: hashpartitioning(i_item_id#40, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(63) ColumnarToRow [codegen id : 6] +Input [2]: [i_item_id#40, sum#42] + +(64) 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 +(65) Union -(60) HashAggregate [codegen id : 7] +(66) 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 +(67) RowToColumnar +Input [3]: [i_item_id#12, sum#47, isEmpty#48] + +(68) CometColumnarExchange +Input [3]: [i_item_id#12, sum#47, isEmpty#48] +Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(69) ColumnarToRow [codegen id : 8] 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 : 8] +(70) 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 +(71) TakeOrderedAndProject 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 ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (68) -+- * ColumnarToRow (67) - +- CometProject (66) - +- CometFilter (65) - +- CometScan parquet spark_catalog.default.date_dim (64) +BroadcastExchange (76) ++- * ColumnarToRow (75) + +- CometProject (74) + +- CometFilter (73) + +- CometScan parquet spark_catalog.default.date_dim (72) -(64) Scan parquet spark_catalog.default.date_dim +(72) Scan parquet spark_catalog.default.date_dim 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 +(73) 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)) -(66) CometProject +(74) CometProject 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] +(75) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(68) BroadcastExchange +(76) BroadcastExchange Input [1]: [d_date_sk#6] 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#22 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 31 Hosting Expression = cs_sold_date_sk#22 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 48 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 c4ce35e15..e1f157a88 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,89 +1,97 @@ TakeOrderedAndProject [total_sales,i_item_id] WholeStageCodegen (8) HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - InputAdapter - Exchange [i_item_id] #1 - WholeStageCodegen (7) - HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (2) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_item_id] #2 - WholeStageCodegen (1) - HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ca_address_sk] #5 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [i_item_id] #7 - CometProject [i_item_id] - CometFilter [i_item_id,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] #8 - WholeStageCodegen (3) - HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] - CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] - CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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 - 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] #9 - WholeStageCodegen (5) - HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] - CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] - CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #1 + RowToColumnar + WholeStageCodegen (7) + HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (2) + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #2 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [ss_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ca_address_sk] #5 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange [i_item_id] #7 + CometProject [i_item_id] + CometFilter [i_item_id,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] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #8 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [cs_ext_sales_price,i_item_id] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] + CometProject [cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] + CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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 + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #9 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [ws_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] + CometProject [ws_item_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] + CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 + 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 76b1adf1a..324701330 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 @@ -1,49 +1,57 @@ == Physical Plan == -TakeOrderedAndProject (45) -+- * Project (44) - +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (28) - : : +- * Filter (27) - : : +- Window (26) - : : +- * Filter (25) - : : +- Window (24) - : : +- * Sort (23) - : : +- Exchange (22) - : : +- * HashAggregate (21) - : : +- Exchange (20) - : : +- * HashAggregate (19) - : : +- * 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) - : +- * Project (34) - : +- Window (33) - : +- * Sort (32) - : +- Exchange (31) - : +- * HashAggregate (30) - : +- ReusedExchange (29) - +- BroadcastExchange (42) - +- * Project (41) - +- Window (40) - +- * Sort (39) - +- ReusedExchange (38) +TakeOrderedAndProject (53) ++- * Project (52) + +- * BroadcastHashJoin Inner BuildRight (51) + :- * Project (44) + : +- * BroadcastHashJoin Inner BuildRight (43) + : :- * Project (32) + : : +- * Filter (31) + : : +- Window (30) + : : +- * Filter (29) + : : +- Window (28) + : : +- * ColumnarToRow (27) + : : +- CometSort (26) + : : +- CometColumnarExchange (25) + : : +- RowToColumnar (24) + : : +- * HashAggregate (23) + : : +- * ColumnarToRow (22) + : : +- CometColumnarExchange (21) + : : +- RowToColumnar (20) + : : +- * HashAggregate (19) + : : +- * 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 (42) + : +- * Project (41) + : +- Window (40) + : +- * ColumnarToRow (39) + : +- CometSort (38) + : +- CometColumnarExchange (37) + : +- RowToColumnar (36) + : +- * HashAggregate (35) + : +- * ColumnarToRow (34) + : +- ReusedExchange (33) + +- BroadcastExchange (50) + +- * Project (49) + +- Window (48) + +- * ColumnarToRow (47) + +- CometSort (46) + +- ReusedExchange (45) (1) Scan parquet spark_catalog.default.item @@ -140,142 +148,166 @@ 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] -(20) Exchange +(20) RowToColumnar 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=1] -(21) HashAggregate [codegen id : 2] +(21) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(22) ColumnarToRow [codegen id : 2] +Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] + +(23) 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))] 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] -(22) Exchange +(24) RowToColumnar +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] + +(25) CometColumnarExchange 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=2] +Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(23) Sort [codegen id : 3] +(26) CometSort 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 +Arguments: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18], [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] -(24) Window +(27) ColumnarToRow [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] + +(28) 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] -(25) Filter [codegen id : 4] +(29) 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)) -(26) Window +(30) 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] -(27) Filter [codegen id : 13] +(31) 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 : 13] +(32) 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] +(33) ReusedExchange [Reuses operator id: 21] Output [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] -(30) HashAggregate [codegen id : 6] +(34) ColumnarToRow [codegen id : 6] +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] + +(35) 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))] 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#28] -(31) Exchange +(36) RowToColumnar +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] + +(37) CometColumnarExchange 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=3] +Arguments: hashpartitioning(i_category#21, i_brand#22, cc_name#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(32) Sort [codegen id : 7] +(38) CometSort 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 +Arguments: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28], [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] -(33) Window +(39) ColumnarToRow [codegen id : 7] +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] + +(40) Window 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 : 8] +(41) 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 +(42) 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=4] -(36) BroadcastHashJoin [codegen id : 13] +(43) 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 : 13] +(44) 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] +(45) ReusedExchange [Reuses operator id: 37] Output [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] -(39) Sort [codegen id : 11] +(46) CometSort 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 +Arguments: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35], [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] -(40) Window +(47) ColumnarToRow [codegen id : 11] +Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] + +(48) Window 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 : 12] +(49) 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 +(50) 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=5] -(43) BroadcastHashJoin [codegen id : 13] +(51) 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 : 13] +(52) 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] -(45) TakeOrderedAndProject +(53) 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] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = cs_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (49) -+- * ColumnarToRow (48) - +- CometFilter (47) - +- CometScan parquet spark_catalog.default.date_dim (46) +BroadcastExchange (57) ++- * ColumnarToRow (56) + +- CometFilter (55) + +- CometScan parquet spark_catalog.default.date_dim (54) -(46) Scan parquet spark_catalog.default.date_dim +(54) 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 -(47) CometFilter +(55) 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)) -(48) ColumnarToRow [codegen id : 1] +(56) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -(49) BroadcastExchange +(57) 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=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 b64c2efdc..aca14449c 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 @@ -13,41 +13,45 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (3) - Sort [i_category,i_brand,cc_name,d_year,d_moy] + ColumnarToRow InputAdapter - Exchange [i_category,i_brand,cc_name] #1 - 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 (1) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometFilter [i_item_sk,i_brand,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 - CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,cc_name] #1 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_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_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 + CometFilter [cc_call_center_sk,cc_name] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter BroadcastExchange #7 WholeStageCodegen (8) @@ -55,13 +59,16 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (7) - Sort [i_category,i_brand,cc_name,d_year,d_moy] + ColumnarToRow InputAdapter - 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 + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,cc_name] #8 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] + ColumnarToRow + InputAdapter + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 InputAdapter BroadcastExchange #9 WholeStageCodegen (12) @@ -69,6 +76,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (11) - Sort [i_category,i_brand,cc_name,d_year,d_moy] + ColumnarToRow InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + 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 2c82f91e9..f00bd9e4b 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,57 +1,63 @@ == Physical Plan == -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) +TakeOrderedAndProject (59) ++- * Project (58) + +- * BroadcastHashJoin Inner BuildRight (57) + :- * Project (40) + : +- * BroadcastHashJoin Inner BuildRight (39) + : :- * Filter (22) + : : +- * HashAggregate (21) + : : +- * ColumnarToRow (20) + : : +- CometColumnarExchange (19) + : : +- RowToColumnar (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 (38) + : +- * Filter (37) + : +- * HashAggregate (36) + : +- * ColumnarToRow (35) + : +- CometColumnarExchange (34) + : +- RowToColumnar (33) + : +- * HashAggregate (32) + : +- * ColumnarToRow (31) + : +- CometProject (30) + : +- CometBroadcastHashJoin (29) + : :- CometProject (27) + : : +- CometBroadcastHashJoin (26) + : : :- CometFilter (24) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (23) + : : +- ReusedExchange (25) + : +- ReusedExchange (28) + +- BroadcastExchange (56) + +- * Filter (55) + +- * HashAggregate (54) + +- * ColumnarToRow (53) + +- CometColumnarExchange (52) + +- RowToColumnar (51) + +- * HashAggregate (50) + +- * ColumnarToRow (49) + +- CometProject (48) + +- CometBroadcastHashJoin (47) + :- CometProject (45) + : +- CometBroadcastHashJoin (44) + : :- CometFilter (42) + : : +- CometScan parquet spark_catalog.default.web_sales (41) + : +- ReusedExchange (43) + +- ReusedExchange (46) (1) Scan parquet spark_catalog.default.store_sales @@ -101,7 +107,7 @@ ReadSchema: struct Input [2]: [d_date_sk#7, d_date#8] Condition : isnotnull(d_date_sk#7) -(10) ReusedExchange [Reuses operator id: 59] +(10) ReusedExchange [Reuses operator id: 65] Output [1]: [d_date#9] (11) CometBroadcastHashJoin @@ -136,22 +142,28 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] Aggregate Attributes [1]: [sum#10] Results [2]: [i_item_id#6, sum#11] -(18) Exchange +(18) RowToColumnar Input [2]: [i_item_id#6, sum#11] -Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(19) HashAggregate [codegen id : 6] +(19) CometColumnarExchange +Input [2]: [i_item_id#6, sum#11] +Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(20) ColumnarToRow [codegen id : 6] +Input [2]: [i_item_id#6, sum#11] + +(21) 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))#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] -(20) Filter [codegen id : 6] +(22) Filter [codegen id : 6] Input [2]: [item_id#13, ss_item_rev#14] Condition : isnotnull(ss_item_rev#14) -(21) Scan parquet spark_catalog.default.catalog_sales +(23) 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 [] @@ -159,74 +171,80 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(22) CometFilter +(24) 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] +(25) ReusedExchange [Reuses operator id: 5] Output [2]: [i_item_sk#19, i_item_id#20] -(24) CometBroadcastHashJoin +(26) 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 +(27) 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] +(28) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#21] -(27) CometBroadcastHashJoin +(29) 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 +(30) 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] +(31) ColumnarToRow [codegen id : 2] Input [2]: [cs_ext_sales_price#16, i_item_id#20] -(30) HashAggregate [codegen id : 2] +(32) 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 +(33) RowToColumnar +Input [2]: [i_item_id#20, sum#23] + +(34) CometColumnarExchange Input [2]: [i_item_id#20, sum#23] -Arguments: hashpartitioning(i_item_id#20, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(i_item_id#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(32) HashAggregate [codegen id : 3] +(35) ColumnarToRow [codegen id : 3] +Input [2]: [i_item_id#20, sum#23] + +(36) 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] +(37) Filter [codegen id : 3] Input [2]: [item_id#25, cs_item_rev#26] Condition : isnotnull(cs_item_rev#26) -(34) BroadcastExchange +(38) 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] +(39) BroadcastHashJoin [codegen id : 6] Left keys [1]: [item_id#13] Right keys [1]: [item_id#25] Join type: Inner 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))) -(36) Project [codegen id : 6] +(40) 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] -(37) Scan parquet spark_catalog.default.web_sales +(41) 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 [] @@ -234,167 +252,173 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(38) CometFilter +(42) 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] +(43) ReusedExchange [Reuses operator id: 5] Output [2]: [i_item_sk#31, i_item_id#32] -(40) CometBroadcastHashJoin +(44) 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 +(45) 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] +(46) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#33] -(43) CometBroadcastHashJoin +(47) 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 +(48) 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] +(49) ColumnarToRow [codegen id : 4] Input [2]: [ws_ext_sales_price#28, i_item_id#32] -(46) HashAggregate [codegen id : 4] +(50) 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 +(51) RowToColumnar +Input [2]: [i_item_id#32, sum#35] + +(52) CometColumnarExchange +Input [2]: [i_item_id#32, sum#35] +Arguments: hashpartitioning(i_item_id#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(53) ColumnarToRow [codegen id : 5] 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] +(54) 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] +(55) Filter [codegen id : 5] Input [2]: [item_id#37, ws_item_rev#38] Condition : isnotnull(ws_item_rev#38) -(50) BroadcastExchange +(56) 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] +(57) BroadcastHashJoin [codegen id : 6] Left keys [1]: [item_id#13] Right keys [1]: [item_id#37] Join type: Inner 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))) -(52) Project [codegen id : 6] +(58) 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] -(53) TakeOrderedAndProject +(59) 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 (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 +BroadcastExchange (69) ++- * ColumnarToRow (68) + +- CometProject (67) + +- CometBroadcastHashJoin (66) + :- CometFilter (61) + : +- CometScan parquet spark_catalog.default.date_dim (60) + +- CometBroadcastExchange (65) + +- CometProject (64) + +- CometFilter (63) + +- CometScan parquet spark_catalog.default.date_dim (62) + + +(60) 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 -(55) CometFilter +(61) CometFilter Input [2]: [d_date_sk#7, d_date#8] Condition : isnotnull(d_date_sk#7) -(56) Scan parquet spark_catalog.default.date_dim +(62) 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 -(57) CometFilter +(63) 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])) -(58) CometProject +(64) CometProject Input [2]: [d_date#9, d_week_seq#43] Arguments: [d_date#9], [d_date#9] -(59) CometBroadcastExchange +(65) CometBroadcastExchange Input [1]: [d_date#9] Arguments: [d_date#9] -(60) CometBroadcastHashJoin +(66) 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 -(61) CometProject +(67) CometProject Input [2]: [d_date_sk#7, d_date#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(62) ColumnarToRow [codegen id : 1] +(68) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(63) BroadcastExchange +(69) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 57 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:2 Hosting operator id = 63 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -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) +Subquery:3 Hosting operator id = 62 Hosting Expression = Subquery scalar-subquery#44, [id=#45] +* ColumnarToRow (73) ++- CometProject (72) + +- CometFilter (71) + +- CometScan parquet spark_catalog.default.date_dim (70) -(64) Scan parquet spark_catalog.default.date_dim +(70) 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 -(65) CometFilter +(71) CometFilter Input [2]: [d_date#46, d_week_seq#47] Condition : (isnotnull(d_date#46) AND (d_date#46 = 2000-01-03)) -(66) CometProject +(72) CometProject Input [2]: [d_date#46, d_week_seq#47] Arguments: [d_week_seq#47], [d_week_seq#47] -(67) ColumnarToRow [codegen id : 1] +(73) ColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#47] -Subquery:4 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#4 +Subquery:4 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#4 -Subquery:5 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#4 +Subquery:5 Hosting operator id = 41 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 0f5a783b9..f82461dba 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 @@ -6,85 +6,91 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev BroadcastHashJoin [item_id,item_id,ss_item_rev,cs_item_rev] Filter [ss_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 (1) - HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,d_date_sk] - CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id] - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_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 (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] - CometFilter [d_date,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,d_week_seq] - CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [i_item_sk,i_item_id] #4 - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date] #3 + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #1 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [ss_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,d_date_sk] + CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_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 (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date] #3 + CometProject [d_date] + CometFilter [d_date,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,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometBroadcastExchange [i_item_sk,i_item_id] #4 + CometFilter [i_item_sk,i_item_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + ReusedExchange [d_date] #3 InputAdapter 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] #7 - WholeStageCodegen (2) - HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,d_date_sk] - CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id] - CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [i_item_sk,i_item_id] #4 - ReusedExchange [d_date_sk] #5 + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #7 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [cs_ext_sales_price,i_item_id] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,d_date_sk] + CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [i_item_sk,i_item_id] #4 + ReusedExchange [d_date_sk] #5 InputAdapter 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] #9 - WholeStageCodegen (4) - HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,d_date_sk] - CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id] - CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [i_item_sk,i_item_id] #4 - ReusedExchange [d_date_sk] #5 + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #9 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [ws_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,d_date_sk] + CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + 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 afa9351c3..8829b74e0 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,47 +1,50 @@ == Physical Plan == -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) +TakeOrderedAndProject (46) ++- * Project (45) + +- * BroadcastHashJoin Inner BuildRight (44) + :- * Project (26) + : +- * BroadcastHashJoin Inner BuildRight (25) + : :- * Project (19) + : : +- * BroadcastHashJoin Inner BuildRight (18) + : : :- * HashAggregate (13) + : : : +- * ColumnarToRow (12) + : : : +- CometColumnarExchange (11) + : : : +- RowToColumnar (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 (17) + : : +- * ColumnarToRow (16) + : : +- CometFilter (15) + : : +- CometScan parquet spark_catalog.default.store (14) + : +- BroadcastExchange (24) + : +- * ColumnarToRow (23) + : +- CometProject (22) + : +- CometFilter (21) + : +- CometScan parquet spark_catalog.default.date_dim (20) + +- BroadcastExchange (43) + +- * Project (42) + +- * BroadcastHashJoin Inner BuildRight (41) + :- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * HashAggregate (29) + : : +- * ColumnarToRow (28) + : : +- ReusedExchange (27) + : +- BroadcastExchange (33) + : +- * ColumnarToRow (32) + : +- CometFilter (31) + : +- CometScan parquet spark_catalog.default.store (30) + +- BroadcastExchange (40) + +- * ColumnarToRow (39) + +- CometProject (38) + +- CometFilter (37) + +- CometScan parquet spark_catalog.default.date_dim (36) (1) Scan parquet spark_catalog.default.store_sales @@ -90,162 +93,171 @@ Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) T 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] -(10) Exchange +(10) RowToColumnar 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=1] -(11) HashAggregate [codegen id : 8] +(11) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(12) ColumnarToRow [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] + +(13) 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] -(12) Scan parquet spark_catalog.default.store +(14) 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 -(13) CometFilter +(15) 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)) -(14) ColumnarToRow [codegen id : 2] +(16) ColumnarToRow [codegen id : 2] Input [3]: [s_store_sk#35, s_store_id#36, s_store_name#37] -(15) BroadcastExchange +(17) 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=2] -(16) BroadcastHashJoin [codegen id : 8] +(18) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_store_sk#1] Right keys [1]: [s_store_sk#35] Join type: Inner Join condition: None -(17) Project [codegen id : 8] +(19) 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] -(18) Scan parquet spark_catalog.default.date_dim +(20) 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 -(19) CometFilter +(21) 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)) -(20) CometProject +(22) CometProject Input [2]: [d_month_seq#38, d_week_seq#39] Arguments: [d_week_seq#39], [d_week_seq#39] -(21) ColumnarToRow [codegen id : 3] +(23) ColumnarToRow [codegen id : 3] Input [1]: [d_week_seq#39] -(22) BroadcastExchange +(24) BroadcastExchange Input [1]: [d_week_seq#39] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(23) BroadcastHashJoin [codegen id : 8] +(25) BroadcastHashJoin [codegen id : 8] Left keys [1]: [d_week_seq#5] Right keys [1]: [d_week_seq#39] Join type: Inner Join condition: None -(24) Project [codegen id : 8] +(26) 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] -(25) ReusedExchange [Reuses operator id: 10] +(27) ReusedExchange [Reuses operator id: 11] Output [9]: [d_week_seq#50, ss_store_sk#51, sum#52, sum#53, sum#54, sum#55, sum#56, sum#57, sum#58] -(26) HashAggregate [codegen id : 7] +(28) ColumnarToRow [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] + +(29) 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] -(27) Scan parquet spark_catalog.default.store +(30) 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 -(28) CometFilter +(31) CometFilter Input [2]: [s_store_sk#68, s_store_id#69] Condition : (isnotnull(s_store_sk#68) AND isnotnull(s_store_id#69)) -(29) ColumnarToRow [codegen id : 5] +(32) ColumnarToRow [codegen id : 5] Input [2]: [s_store_sk#68, s_store_id#69] -(30) BroadcastExchange +(33) BroadcastExchange Input [2]: [s_store_sk#68, s_store_id#69] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(31) BroadcastHashJoin [codegen id : 7] +(34) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ss_store_sk#51] Right keys [1]: [s_store_sk#68] Join type: Inner Join condition: None -(32) Project [codegen id : 7] +(35) 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] -(33) Scan parquet spark_catalog.default.date_dim +(36) 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 -(34) CometFilter +(37) 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)) -(35) CometProject +(38) CometProject Input [2]: [d_month_seq#70, d_week_seq#71] Arguments: [d_week_seq#71], [d_week_seq#71] -(36) ColumnarToRow [codegen id : 6] +(39) ColumnarToRow [codegen id : 6] Input [1]: [d_week_seq#71] -(37) BroadcastExchange +(40) BroadcastExchange Input [1]: [d_week_seq#71] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(38) BroadcastHashJoin [codegen id : 7] +(41) BroadcastHashJoin [codegen id : 7] Left keys [1]: [d_week_seq#50] Right keys [1]: [d_week_seq#71] Join type: Inner Join condition: None -(39) Project [codegen id : 7] +(42) 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] -(40) BroadcastExchange +(43) 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=6] -(41) BroadcastHashJoin [codegen id : 8] +(44) 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 -(42) Project [codegen id : 8] +(45) 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] -(43) TakeOrderedAndProject +(46) 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 58bae46d1..43f903e19 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 @@ -7,19 +7,21 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] BroadcastHashJoin [ss_store_sk,s_store_sk] 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 (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] - ColumnarToRow - InputAdapter - CometProject [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] - CometBroadcastHashJoin [ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_week_seq,d_day_name] - CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #2 - CometFilter [d_date_sk,d_week_seq,d_day_name] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] + ColumnarToRow + InputAdapter + CometColumnarExchange [d_week_seq,ss_store_sk] #1 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometProject [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] + CometBroadcastHashJoin [ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_week_seq,d_day_name] + CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #2 + CometFilter [d_date_sk,d_week_seq,d_day_name] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) @@ -43,8 +45,9 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] BroadcastHashJoin [ss_store_sk,s_store_sk] 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 - ReusedExchange [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] #1 + ColumnarToRow + InputAdapter + ReusedExchange [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] #1 InputAdapter BroadcastExchange #6 WholeStageCodegen (5) 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 46a4bdbac..f8552c4f8 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,44 +1,48 @@ == Physical Plan == -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) +TakeOrderedAndProject (44) ++- * Filter (43) + +- * HashAggregate (42) + +- * ColumnarToRow (41) + +- CometColumnarExchange (40) + +- RowToColumnar (39) + +- * HashAggregate (38) + +- * Project (37) + +- * BroadcastHashJoin Inner BuildRight (36) + :- * 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 (35) + +- * Project (34) + +- * BroadcastHashJoin Inner BuildRight (33) + :- * ColumnarToRow (22) + : +- CometFilter (21) + : +- CometScan parquet spark_catalog.default.item (20) + +- BroadcastExchange (32) + +- * Filter (31) + +- * HashAggregate (30) + +- * ColumnarToRow (29) + +- CometColumnarExchange (28) + +- RowToColumnar (27) + +- * HashAggregate (26) + +- * ColumnarToRow (25) + +- CometFilter (24) + +- CometScan parquet spark_catalog.default.item (23) (1) Scan parquet spark_catalog.default.customer_address @@ -167,152 +171,162 @@ 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 +(27) RowToColumnar 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] +(28) CometColumnarExchange +Input [3]: [i_category#17, sum#20, count#21] +Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(29) ColumnarToRow [codegen id : 2] +Input [3]: [i_category#17, sum#20, count#21] + +(30) 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] +(31) Filter [codegen id : 2] Input [2]: [avg(i_current_price)#23, i_category#17] Condition : isnotnull(avg(i_current_price)#23) -(30) BroadcastExchange +(32) 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] +(33) 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#14 as decimal(14,7)) > (1.2 * avg(i_current_price)#23)) -(32) Project [codegen id : 3] +(34) 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] -(33) BroadcastExchange +(35) BroadcastExchange Input [1]: [i_item_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(34) BroadcastHashJoin [codegen id : 4] +(36) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_item_sk#5] Right keys [1]: [i_item_sk#13] Join type: Inner Join condition: None -(35) Project [codegen id : 4] +(37) Project [codegen id : 4] Output [1]: [ca_state#2] Input [3]: [ca_state#2, ss_item_sk#5, i_item_sk#13] -(36) HashAggregate [codegen id : 4] +(38) HashAggregate [codegen id : 4] Input [1]: [ca_state#2] Keys [1]: [ca_state#2] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#24] Results [2]: [ca_state#2, count#25] -(37) Exchange +(39) RowToColumnar +Input [2]: [ca_state#2, count#25] + +(40) CometColumnarExchange Input [2]: [ca_state#2, count#25] -Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(38) HashAggregate [codegen id : 5] +(41) ColumnarToRow [codegen id : 5] +Input [2]: [ca_state#2, count#25] + +(42) 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)#26] Results [2]: [ca_state#2 AS state#27, count(1)#26 AS cnt#28] -(39) Filter [codegen id : 5] +(43) Filter [codegen id : 5] Input [2]: [state#27, cnt#28] Condition : (cnt#28 >= 10) -(40) TakeOrderedAndProject +(44) TakeOrderedAndProject Input [2]: [state#27, cnt#28] Arguments: 100, [cnt#28 ASC NULLS FIRST], [state#27, cnt#28] ===== Subqueries ===== 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) +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan parquet spark_catalog.default.date_dim (45) -(41) Scan parquet spark_catalog.default.date_dim +(45) 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 -(42) CometFilter +(46) 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)) -(43) CometProject +(47) CometProject Input [2]: [d_date_sk#9, d_month_seq#10] Arguments: [d_date_sk#9], [d_date_sk#9] -(44) ColumnarToRow [codegen id : 1] +(48) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(45) BroadcastExchange +(49) BroadcastExchange Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -Subquery:2 Hosting operator id = 42 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] +Subquery:2 Hosting operator id = 46 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] -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) +Subquery:3 Hosting operator id = 45 Hosting Expression = Subquery scalar-subquery#11, [id=#12] +* ColumnarToRow (56) ++- CometHashAggregate (55) + +- CometColumnarExchange (54) + +- CometHashAggregate (53) + +- CometProject (52) + +- CometFilter (51) + +- CometScan parquet spark_catalog.default.date_dim (50) -(46) Scan parquet spark_catalog.default.date_dim +(50) 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 -(47) CometFilter +(51) 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)) -(48) CometProject +(52) CometProject Input [3]: [d_month_seq#29, d_year#30, d_moy#31] Arguments: [d_month_seq#29], [d_month_seq#29] -(49) CometHashAggregate +(53) CometHashAggregate Input [1]: [d_month_seq#29] Keys [1]: [d_month_seq#29] Functions: [] -(50) ColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#29] - -(51) Exchange +(54) CometColumnarExchange Input [1]: [d_month_seq#29] -Arguments: hashpartitioning(d_month_seq#29, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Arguments: hashpartitioning(d_month_seq#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(52) HashAggregate [codegen id : 2] +(55) CometHashAggregate Input [1]: [d_month_seq#29] Keys [1]: [d_month_seq#29] Functions: [] -Aggregate Attributes: [] -Results [1]: [d_month_seq#29] + +(56) ColumnarToRow [codegen id : 1] +Input [1]: [d_month_seq#29] Subquery:4 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-spark4_0/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/simplified.txt index febbfd32f..e28908700 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 @@ -2,74 +2,76 @@ TakeOrderedAndProject [cnt,state] WholeStageCodegen (5) Filter [cnt] HashAggregate [ca_state,count] [count(1),state,cnt,count] - InputAdapter - Exchange [ca_state] #1 - WholeStageCodegen (4) - HashAggregate [ca_state] [count,count] - Project [ca_state] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometProject [ca_state,ss_item_sk] - CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] - CometProject [ca_state,ss_item_sk,ss_sold_date_sk] - CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] - CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_customer_sk,ss_sold_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 #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - 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 + ColumnarToRow + InputAdapter + CometColumnarExchange [ca_state] #1 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [ca_state] [count,count] + Project [ca_state] + BroadcastHashJoin [ss_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometProject [ca_state,ss_item_sk] + CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] + CometProject [ca_state,ss_item_sk,ss_sold_date_sk] + CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] + CometProject [ca_state,c_customer_sk] + CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_customer_sk,ss_sold_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 #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + ReusedSubquery [d_month_seq] #2 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + Subquery #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [d_month_seq] + CometColumnarExchange [d_month_seq] #5 CometHashAggregate [d_month_seq] CometProject [d_month_seq] CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - 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 #7 - WholeStageCodegen (3) - Project [i_item_sk] - BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_current_price,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - InputAdapter - 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] #9 - WholeStageCodegen (1) - HashAggregate [i_category,i_current_price] [sum,count,sum,count] - ColumnarToRow - InputAdapter - CometFilter [i_current_price,i_category] - CometScan parquet spark_catalog.default.item [i_current_price,i_category] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + 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 #7 + WholeStageCodegen (3) + Project [i_item_sk] + BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_current_price,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] + InputAdapter + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category] #9 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [i_category,i_current_price] [sum,count,sum,count] + ColumnarToRow + InputAdapter + CometFilter [i_current_price,i_category] + CometScan parquet spark_catalog.default.item [i_current_price,i_category] 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 342a3deb8..1f6d378e8 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 @@ -1,67 +1,75 @@ == Physical Plan == -TakeOrderedAndProject (63) -+- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- Union (59) - :- * HashAggregate (28) - : +- Exchange (27) - : +- * HashAggregate (26) - : +- * 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) - : +- * 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) - +- * 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) +TakeOrderedAndProject (71) ++- * HashAggregate (70) + +- * ColumnarToRow (69) + +- CometColumnarExchange (68) + +- RowToColumnar (67) + +- * HashAggregate (66) + +- Union (65) + :- * HashAggregate (30) + : +- * ColumnarToRow (29) + : +- CometColumnarExchange (28) + : +- RowToColumnar (27) + : +- * HashAggregate (26) + : +- * 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 (47) + : +- * ColumnarToRow (46) + : +- CometColumnarExchange (45) + : +- RowToColumnar (44) + : +- * HashAggregate (43) + : +- * ColumnarToRow (42) + : +- CometProject (41) + : +- CometBroadcastHashJoin (40) + : :- CometProject (38) + : : +- CometBroadcastHashJoin (37) + : : :- CometProject (35) + : : : +- CometBroadcastHashJoin (34) + : : : :- CometFilter (32) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (31) + : : : +- ReusedExchange (33) + : : +- ReusedExchange (36) + : +- ReusedExchange (39) + +- * HashAggregate (64) + +- * ColumnarToRow (63) + +- CometColumnarExchange (62) + +- RowToColumnar (61) + +- * HashAggregate (60) + +- * ColumnarToRow (59) + +- CometProject (58) + +- CometBroadcastHashJoin (57) + :- CometProject (55) + : +- CometBroadcastHashJoin (54) + : :- CometProject (52) + : : +- CometBroadcastHashJoin (51) + : : :- CometFilter (49) + : : : +- CometScan parquet spark_catalog.default.web_sales (48) + : : +- ReusedExchange (50) + : +- ReusedExchange (53) + +- ReusedExchange (56) (1) Scan parquet spark_catalog.default.store_sales @@ -190,18 +198,24 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] Aggregate Attributes [1]: [sum#15] Results [2]: [i_item_id#12, sum#16] -(27) Exchange +(27) RowToColumnar Input [2]: [i_item_id#12, sum#16] -Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(28) HashAggregate [codegen id : 2] +(28) CometColumnarExchange +Input [2]: [i_item_id#12, sum#16] +Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(29) ColumnarToRow [codegen id : 2] +Input [2]: [i_item_id#12, sum#16] + +(30) 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))#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 +(31) Scan parquet spark_catalog.default.catalog_sales 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 [] @@ -209,68 +223,74 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#22), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(30) CometFilter +(32) CometFilter 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] +(33) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#24] -(32) CometBroadcastHashJoin +(34) 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 +(35) 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] +(36) ReusedExchange [Reuses operator id: 12] Output [1]: [ca_address_sk#25] -(35) CometBroadcastHashJoin +(37) 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 +(38) 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] +(39) ReusedExchange [Reuses operator id: 22] Output [2]: [i_item_sk#26, i_item_id#27] -(38) CometBroadcastHashJoin +(40) 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 +(41) 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] +(42) ColumnarToRow [codegen id : 3] Input [2]: [cs_ext_sales_price#21, i_item_id#27] -(41) HashAggregate [codegen id : 3] +(43) 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 +(44) RowToColumnar +Input [2]: [i_item_id#27, sum#29] + +(45) CometColumnarExchange +Input [2]: [i_item_id#27, sum#29] +Arguments: hashpartitioning(i_item_id#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(46) ColumnarToRow [codegen id : 4] Input [2]: [i_item_id#27, sum#29] -Arguments: hashpartitioning(i_item_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(43) HashAggregate [codegen id : 4] +(47) 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 +(48) Scan parquet spark_catalog.default.web_sales 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 [] @@ -278,125 +298,137 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(45) CometFilter +(49) CometFilter 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] +(50) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#37] -(47) CometBroadcastHashJoin +(51) 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 +(52) 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] +(53) ReusedExchange [Reuses operator id: 12] Output [1]: [ca_address_sk#38] -(50) CometBroadcastHashJoin +(54) 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 +(55) 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] +(56) ReusedExchange [Reuses operator id: 22] Output [2]: [i_item_sk#39, i_item_id#40] -(53) CometBroadcastHashJoin +(57) 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 +(58) 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] +(59) ColumnarToRow [codegen id : 5] Input [2]: [ws_ext_sales_price#34, i_item_id#40] -(56) HashAggregate [codegen id : 5] +(60) 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 +(61) RowToColumnar Input [2]: [i_item_id#40, sum#42] -Arguments: hashpartitioning(i_item_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(58) HashAggregate [codegen id : 6] +(62) CometColumnarExchange +Input [2]: [i_item_id#40, sum#42] +Arguments: hashpartitioning(i_item_id#40, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(63) ColumnarToRow [codegen id : 6] +Input [2]: [i_item_id#40, sum#42] + +(64) 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 +(65) Union -(60) HashAggregate [codegen id : 7] +(66) 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 +(67) RowToColumnar +Input [3]: [i_item_id#12, sum#47, isEmpty#48] + +(68) CometColumnarExchange +Input [3]: [i_item_id#12, sum#47, isEmpty#48] +Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(69) ColumnarToRow [codegen id : 8] 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 : 8] +(70) 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 +(71) TakeOrderedAndProject 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 ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (68) -+- * ColumnarToRow (67) - +- CometProject (66) - +- CometFilter (65) - +- CometScan parquet spark_catalog.default.date_dim (64) +BroadcastExchange (76) ++- * ColumnarToRow (75) + +- CometProject (74) + +- CometFilter (73) + +- CometScan parquet spark_catalog.default.date_dim (72) -(64) Scan parquet spark_catalog.default.date_dim +(72) Scan parquet spark_catalog.default.date_dim 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 +(73) 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)) -(66) CometProject +(74) CometProject 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] +(75) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(68) BroadcastExchange +(76) BroadcastExchange Input [1]: [d_date_sk#6] 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#22 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 31 Hosting Expression = cs_sold_date_sk#22 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 48 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 51025cf34..6c079c1e4 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,89 +1,97 @@ TakeOrderedAndProject [i_item_id,total_sales] WholeStageCodegen (8) HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - InputAdapter - Exchange [i_item_id] #1 - WholeStageCodegen (7) - HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (2) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_item_id] #2 - WholeStageCodegen (1) - HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ca_address_sk] #5 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [i_item_id] #7 - CometProject [i_item_id] - CometFilter [i_item_id,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] #8 - WholeStageCodegen (3) - HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] - CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] - CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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 - 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] #9 - WholeStageCodegen (5) - HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] - CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] - CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #1 + RowToColumnar + WholeStageCodegen (7) + HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (2) + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #2 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [ss_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ca_address_sk] #5 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange [i_item_id] #7 + CometProject [i_item_id] + CometFilter [i_item_id,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] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #8 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [cs_ext_sales_price,i_item_id] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] + CometProject [cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] + CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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 + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #9 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [ws_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] + CometProject [ws_item_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] + CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 + 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 6d0d03e13..61610b6ee 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,69 +1,73 @@ == Physical Plan == -* 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) +* Project (69) ++- * BroadcastNestedLoopJoin Inner BuildRight (68) + :- * HashAggregate (43) + : +- * ColumnarToRow (42) + : +- CometColumnarExchange (41) + : +- RowToColumnar (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 (67) + +- * HashAggregate (66) + +- * ColumnarToRow (65) + +- CometColumnarExchange (64) + +- RowToColumnar (63) + +- * HashAggregate (62) + +- * ColumnarToRow (61) + +- CometProject (60) + +- CometBroadcastHashJoin (59) + :- CometProject (57) + : +- CometBroadcastHashJoin (56) + : :- CometProject (54) + : : +- CometBroadcastHashJoin (53) + : : :- CometProject (51) + : : : +- CometBroadcastHashJoin (50) + : : : :- CometProject (48) + : : : : +- CometBroadcastHashJoin (47) + : : : : :- CometFilter (45) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (44) + : : : : +- ReusedExchange (46) + : : : +- ReusedExchange (49) + : : +- ReusedExchange (52) + : +- ReusedExchange (55) + +- ReusedExchange (58) (1) Scan parquet spark_catalog.default.store_sales @@ -252,18 +256,24 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] Aggregate Attributes [1]: [sum#23] Results [1]: [sum#24] -(40) Exchange +(40) RowToColumnar Input [1]: [sum#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] -(41) HashAggregate [codegen id : 4] +(41) CometColumnarExchange +Input [1]: [sum#24] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(42) ColumnarToRow [codegen id : 4] +Input [1]: [sum#24] + +(43) 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))#25] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#25,17,2) AS promotions#26] -(42) Scan parquet spark_catalog.default.store_sales +(44) 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 [] @@ -271,135 +281,141 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#31), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(43) CometFilter +(45) 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)) -(44) ReusedExchange [Reuses operator id: 6] +(46) ReusedExchange [Reuses operator id: 6] Output [1]: [s_store_sk#33] -(45) CometBroadcastHashJoin +(47) 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 -(46) CometProject +(48) 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] -(47) ReusedExchange [Reuses operator id: 18] +(49) ReusedExchange [Reuses operator id: 18] Output [1]: [d_date_sk#34] -(48) CometBroadcastHashJoin +(50) 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 -(49) CometProject +(51) 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] -(50) ReusedExchange [Reuses operator id: 23] +(52) ReusedExchange [Reuses operator id: 23] Output [2]: [c_customer_sk#35, c_current_addr_sk#36] -(51) CometBroadcastHashJoin +(53) 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 -(52) CometProject +(54) 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] -(53) ReusedExchange [Reuses operator id: 29] +(55) ReusedExchange [Reuses operator id: 29] Output [1]: [ca_address_sk#37] -(54) CometBroadcastHashJoin +(56) 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 -(55) CometProject +(57) 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] -(56) ReusedExchange [Reuses operator id: 35] +(58) ReusedExchange [Reuses operator id: 35] Output [1]: [i_item_sk#38] -(57) CometBroadcastHashJoin +(59) 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 -(58) CometProject +(60) 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] -(59) ColumnarToRow [codegen id : 2] +(61) ColumnarToRow [codegen id : 2] Input [1]: [ss_ext_sales_price#30] -(60) HashAggregate [codegen id : 2] +(62) HashAggregate [codegen id : 2] Input [1]: [ss_ext_sales_price#30] Keys: [] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#30))] Aggregate Attributes [1]: [sum#39] Results [1]: [sum#40] -(61) Exchange +(63) RowToColumnar +Input [1]: [sum#40] + +(64) CometColumnarExchange +Input [1]: [sum#40] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(65) ColumnarToRow [codegen id : 3] Input [1]: [sum#40] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] -(62) HashAggregate [codegen id : 3] +(66) HashAggregate [codegen id : 3] Input [1]: [sum#40] Keys: [] 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] -(63) BroadcastExchange +(67) BroadcastExchange Input [1]: [total#42] Arguments: IdentityBroadcastMode, [plan_id=3] -(64) BroadcastNestedLoopJoin [codegen id : 4] +(68) BroadcastNestedLoopJoin [codegen id : 4] Join type: Inner Join condition: None -(65) Project [codegen id : 4] +(69) 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 (70) -+- * ColumnarToRow (69) - +- CometProject (68) - +- CometFilter (67) - +- CometScan parquet spark_catalog.default.date_dim (66) +BroadcastExchange (74) ++- * ColumnarToRow (73) + +- CometProject (72) + +- CometFilter (71) + +- CometScan parquet spark_catalog.default.date_dim (70) -(66) Scan parquet spark_catalog.default.date_dim +(70) 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 -(67) CometFilter +(71) 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)) -(68) CometProject +(72) CometProject Input [3]: [d_date_sk#14, d_year#15, d_moy#16] Arguments: [d_date_sk#14], [d_date_sk#14] -(69) ColumnarToRow [codegen id : 1] +(73) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(70) BroadcastExchange +(74) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -Subquery:2 Hosting operator id = 42 Hosting Expression = ss_sold_date_sk#31 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 44 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 8d39d9e4e..e38989b41 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 @@ -2,82 +2,86 @@ WholeStageCodegen (4) Project [promotions,total] BroadcastNestedLoopJoin HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),promotions,sum] - InputAdapter - Exchange #1 - WholeStageCodegen (1) - HashAggregate [ss_ext_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] - CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk] - CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk] - CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,p_promo_sk] - CometProject [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk] #3 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_gmt_offset] - CometScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] - CometBroadcastExchange [p_promo_sk] #4 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #6 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk] #7 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk] #8 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_category] + ColumnarToRow + InputAdapter + CometColumnarExchange #1 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [ss_ext_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk] + CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,p_promo_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk] #3 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_gmt_offset] + CometScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] + CometBroadcastExchange [p_promo_sk] #4 + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] + CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #6 + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange [ca_address_sk] #7 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometBroadcastExchange [i_item_sk] #8 + CometProject [i_item_sk] + CometFilter [i_item_sk,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_category] InputAdapter BroadcastExchange #9 WholeStageCodegen (3) HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),total,sum] - InputAdapter - Exchange #10 - WholeStageCodegen (2) - HashAggregate [ss_ext_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] - CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk] - CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk] - CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_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 - 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 + ColumnarToRow + InputAdapter + CometColumnarExchange #10 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [ss_ext_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk] + CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_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 + 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 cba9fc4b9..e804b1dd3 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,32 +1,34 @@ == Physical Plan == -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) +TakeOrderedAndProject (30) ++- * HashAggregate (29) + +- * ColumnarToRow (28) + +- CometColumnarExchange (27) + +- RowToColumnar (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 @@ -150,18 +152,24 @@ Functions [5]: [partial_sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) < 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] -(26) Exchange +(26) RowToColumnar 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=1] -(27) HashAggregate [codegen id : 2] +(27) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(28) ColumnarToRow [codegen id : 2] +Input [8]: [_groupingexpression#14, sm_type#9, web_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] + +(29) 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] -(28) TakeOrderedAndProject +(30) 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 b9e35e282..9cd3afd85 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,32 +1,34 @@ 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 (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 (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] - ColumnarToRow - InputAdapter - CometProject [w_warehouse_name] [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,_groupingexpression] - CometBroadcastHashJoin [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name,d_date_sk] - CometProject [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name] - CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_site_sk,web_name] - CometProject [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type] - CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] - CometProject [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name] - CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name] - CometFilter [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_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] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 - CometFilter [sm_ship_mode_sk,sm_type] - CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange [web_site_sk,web_name] #4 - CometFilter [web_site_sk,web_name] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_name] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + ColumnarToRow + InputAdapter + CometColumnarExchange [_groupingexpression,sm_type,web_name] #1 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometProject [w_warehouse_name] [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,_groupingexpression] + CometBroadcastHashJoin [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name,d_date_sk] + CometProject [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_site_sk,web_name] + CometProject [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] + CometProject [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_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] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 + CometFilter [w_warehouse_sk,w_warehouse_name] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 + CometFilter [sm_ship_mode_sk,sm_type] + CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] + CometBroadcastExchange [web_site_sk,web_name] #4 + CometFilter [web_site_sk,web_name] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_name] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + 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 c10d628ae..53b396c29 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,33 +1,37 @@ == Physical Plan == -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) +TakeOrderedAndProject (33) ++- * Project (32) + +- * Filter (31) + +- Window (30) + +- * ColumnarToRow (29) + +- CometSort (28) + +- CometColumnarExchange (27) + +- RowToColumnar (26) + +- * HashAggregate (25) + +- * ColumnarToRow (24) + +- CometColumnarExchange (23) + +- RowToColumnar (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 @@ -132,70 +136,82 @@ Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] Aggregate Attributes [1]: [sum#19] Results [3]: [i_manager_id#5, d_moy#17, sum#20] -(22) Exchange +(22) RowToColumnar 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] -(23) HashAggregate [codegen id : 2] +(23) CometColumnarExchange +Input [3]: [i_manager_id#5, d_moy#17, sum#20] +Arguments: hashpartitioning(i_manager_id#5, d_moy#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(24) ColumnarToRow [codegen id : 2] +Input [3]: [i_manager_id#5, d_moy#17, sum#20] + +(25) 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))#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] -(24) Exchange +(26) RowToColumnar +Input [3]: [i_manager_id#5, sum_sales#22, _w0#23] + +(27) CometColumnarExchange +Input [3]: [i_manager_id#5, sum_sales#22, _w0#23] +Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(28) CometSort Input [3]: [i_manager_id#5, sum_sales#22, _w0#23] -Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: [i_manager_id#5, sum_sales#22, _w0#23], [i_manager_id#5 ASC NULLS FIRST] -(25) Sort [codegen id : 3] +(29) ColumnarToRow [codegen id : 3] Input [3]: [i_manager_id#5, sum_sales#22, _w0#23] -Arguments: [i_manager_id#5 ASC NULLS FIRST], false, 0 -(26) Window +(30) 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] -(27) Filter [codegen id : 4] +(31) 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 -(28) Project [codegen id : 4] +(32) 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] -(29) TakeOrderedAndProject +(33) 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 = 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) +BroadcastExchange (38) ++- * ColumnarToRow (37) + +- CometProject (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.date_dim (34) -(30) Scan parquet spark_catalog.default.date_dim +(34) 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 -(31) CometFilter +(35) 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)) -(32) CometProject +(36) 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] -(33) ColumnarToRow [codegen id : 1] +(37) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#15, d_moy#17] -(34) BroadcastExchange +(38) 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 e84d3c2a0..34a22f010 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 @@ -5,41 +5,45 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_manager_id] WholeStageCodegen (3) - Sort [i_manager_id] + ColumnarToRow InputAdapter - Exchange [i_manager_id] #1 - 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 (1) - HashAggregate [i_manager_id,d_moy,ss_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [i_manager_id,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] - CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] - CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_item_sk,i_manager_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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_date_sk,d_month_seq,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] + CometSort [i_manager_id,sum_sales,_w0] + CometColumnarExchange [i_manager_id] #1 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [i_manager_id,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_manager_id,d_moy] #2 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [i_manager_id,d_moy,ss_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [i_manager_id,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] + CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] + CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_item_sk,i_manager_id] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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_date_sk,d_month_seq,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometBroadcastExchange [d_date_sk,d_moy] #5 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_month_seq,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometBroadcastExchange [s_store_sk] #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 d6e68a2ac..cb87fc6fc 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 @@ -1,185 +1,193 @@ == Physical Plan == -* Sort (181) -+- Exchange (180) - +- * Project (179) - +- * SortMergeJoin Inner (178) - :- * Sort (110) - : +- Exchange (109) - : +- * HashAggregate (108) - : +- * HashAggregate (107) - : +- * Project (106) - : +- * BroadcastHashJoin Inner BuildRight (105) - : :- * Project (99) - : : +- * BroadcastHashJoin Inner BuildRight (98) - : : :- * Project (96) - : : : +- * BroadcastHashJoin Inner BuildRight (95) - : : : :- * Project (90) - : : : : +- * BroadcastHashJoin Inner BuildRight (89) - : : : : :- * Project (87) - : : : : : +- * BroadcastHashJoin Inner BuildRight (86) - : : : : : :- * Project (81) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (80) - : : : : : : :- * Project (78) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (77) - : : : : : : : :- * Project (72) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (71) - : : : : : : : : :- * Project (66) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (65) - : : : : : : : : : :- * Project (63) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (62) - : : : : : : : : : : :- * Project (57) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (56) - : : : : : : : : : : : :- * Project (54) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (53) - : : : : : : : : : : : : :- * Project (48) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (47) - : : : : : : : : : : : : : :- * Project (42) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : : : : : : : : : : : :- * Project (36) - : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (35) - : : : : : : : : : : : : : : : :- * Project (33) - : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (32) - : : : : : : : : : : : : : : : : :- * Sort (11) - : : : : : : : : : : : : : : : : : +- Exchange (10) - : : : : : : : : : : : : : : : : : +- * ColumnarToRow (9) - : : : : : : : : : : : : : : : : : +- CometProject (8) - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) - : : : : : : : : : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- CometProject (6) - : : : : : : : : : : : : : : : : : +- CometFilter (5) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : : : : : : : : : : : : : : : +- * Sort (31) - : : : : : : : : : : : : : : : : +- * Project (30) - : : : : : : : : : : : : : : : : +- * Filter (29) - : : : : : : : : : : : : : : : : +- * HashAggregate (28) - : : : : : : : : : : : : : : : : +- Exchange (27) - : : : : : : : : : : : : : : : : +- * HashAggregate (26) - : : : : : : : : : : : : : : : : +- * Project (25) - : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (24) - : : : : : : : : : : : : : : : : :- * Sort (17) - : : : : : : : : : : : : : : : : : +- Exchange (16) - : : : : : : : : : : : : : : : : : +- * ColumnarToRow (15) - : : : : : : : : : : : : : : : : : +- CometProject (14) - : : : : : : : : : : : : : : : : : +- CometFilter (13) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (12) - : : : : : : : : : : : : : : : : +- * Sort (23) - : : : : : : : : : : : : : : : : +- Exchange (22) - : : : : : : : : : : : : : : : : +- * ColumnarToRow (21) - : : : : : : : : : : : : : : : : +- CometProject (20) - : : : : : : : : : : : : : : : : +- CometFilter (19) - : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (18) - : : : : : : : : : : : : : : : +- ReusedExchange (34) - : : : : : : : : : : : : : : +- BroadcastExchange (40) - : : : : : : : : : : : : : : +- * ColumnarToRow (39) - : : : : : : : : : : : : : : +- CometFilter (38) - : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (37) - : : : : : : : : : : : : : +- BroadcastExchange (46) - : : : : : : : : : : : : : +- * ColumnarToRow (45) - : : : : : : : : : : : : : +- CometFilter (44) - : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (43) - : : : : : : : : : : : : +- BroadcastExchange (52) - : : : : : : : : : : : : +- * ColumnarToRow (51) - : : : : : : : : : : : : +- CometFilter (50) - : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (49) - : : : : : : : : : : : +- ReusedExchange (55) - : : : : : : : : : : +- BroadcastExchange (61) - : : : : : : : : : : +- * ColumnarToRow (60) - : : : : : : : : : : +- CometFilter (59) - : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (58) - : : : : : : : : : +- ReusedExchange (64) - : : : : : : : : +- BroadcastExchange (70) - : : : : : : : : +- * ColumnarToRow (69) - : : : : : : : : +- CometFilter (68) - : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (67) - : : : : : : : +- BroadcastExchange (76) - : : : : : : : +- * ColumnarToRow (75) - : : : : : : : +- CometFilter (74) - : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (73) - : : : : : : +- ReusedExchange (79) - : : : : : +- BroadcastExchange (85) - : : : : : +- * ColumnarToRow (84) - : : : : : +- CometFilter (83) - : : : : : +- CometScan parquet spark_catalog.default.customer_address (82) - : : : : +- ReusedExchange (88) - : : : +- BroadcastExchange (94) - : : : +- * ColumnarToRow (93) - : : : +- CometFilter (92) - : : : +- CometScan parquet spark_catalog.default.income_band (91) - : : +- ReusedExchange (97) - : +- BroadcastExchange (104) - : +- * ColumnarToRow (103) - : +- CometProject (102) - : +- CometFilter (101) - : +- CometScan parquet spark_catalog.default.item (100) - +- * Sort (177) - +- Exchange (176) - +- * HashAggregate (175) - +- * HashAggregate (174) - +- * Project (173) - +- * BroadcastHashJoin Inner BuildRight (172) - :- * Project (170) - : +- * BroadcastHashJoin Inner BuildRight (169) - : :- * Project (167) - : : +- * BroadcastHashJoin Inner BuildRight (166) - : : :- * Project (164) - : : : +- * BroadcastHashJoin Inner BuildRight (163) - : : : :- * Project (161) - : : : : +- * BroadcastHashJoin Inner BuildRight (160) - : : : : :- * Project (158) - : : : : : +- * BroadcastHashJoin Inner BuildRight (157) - : : : : : :- * Project (155) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (154) - : : : : : : :- * Project (152) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (151) - : : : : : : : :- * Project (149) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (148) - : : : : : : : : :- * Project (146) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (145) - : : : : : : : : : :- * Project (143) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (142) - : : : : : : : : : : :- * Project (140) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (139) - : : : : : : : : : : : :- * Project (137) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (136) - : : : : : : : : : : : : :- * Project (134) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (133) - : : : : : : : : : : : : : :- * Project (131) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (130) - : : : : : : : : : : : : : : :- * Project (128) - : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (127) - : : : : : : : : : : : : : : : :- * Sort (121) - : : : : : : : : : : : : : : : : +- Exchange (120) - : : : : : : : : : : : : : : : : +- * ColumnarToRow (119) - : : : : : : : : : : : : : : : : +- CometProject (118) - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (117) - : : : : : : : : : : : : : : : : :- CometBroadcastExchange (113) - : : : : : : : : : : : : : : : : : +- CometFilter (112) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (111) - : : : : : : : : : : : : : : : : +- CometProject (116) - : : : : : : : : : : : : : : : : +- CometFilter (115) - : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (114) - : : : : : : : : : : : : : : : +- * Sort (126) - : : : : : : : : : : : : : : : +- * Project (125) - : : : : : : : : : : : : : : : +- * Filter (124) - : : : : : : : : : : : : : : : +- * HashAggregate (123) - : : : : : : : : : : : : : : : +- ReusedExchange (122) - : : : : : : : : : : : : : : +- ReusedExchange (129) - : : : : : : : : : : : : : +- ReusedExchange (132) - : : : : : : : : : : : : +- ReusedExchange (135) - : : : : : : : : : : : +- ReusedExchange (138) - : : : : : : : : : : +- ReusedExchange (141) - : : : : : : : : : +- ReusedExchange (144) - : : : : : : : : +- ReusedExchange (147) - : : : : : : : +- ReusedExchange (150) - : : : : : : +- ReusedExchange (153) - : : : : : +- ReusedExchange (156) - : : : : +- ReusedExchange (159) - : : : +- ReusedExchange (162) - : : +- ReusedExchange (165) - : +- ReusedExchange (168) - +- ReusedExchange (171) +* ColumnarToRow (189) ++- CometSort (188) + +- CometColumnarExchange (187) + +- RowToColumnar (186) + +- * Project (185) + +- * SortMergeJoin Inner (184) + :- * ColumnarToRow (113) + : +- CometSort (112) + : +- CometColumnarExchange (111) + : +- RowToColumnar (110) + : +- * HashAggregate (109) + : +- * HashAggregate (108) + : +- * Project (107) + : +- * BroadcastHashJoin Inner BuildRight (106) + : :- * Project (100) + : : +- * BroadcastHashJoin Inner BuildRight (99) + : : :- * Project (97) + : : : +- * BroadcastHashJoin Inner BuildRight (96) + : : : :- * Project (91) + : : : : +- * BroadcastHashJoin Inner BuildRight (90) + : : : : :- * Project (88) + : : : : : +- * BroadcastHashJoin Inner BuildRight (87) + : : : : : :- * Project (82) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (81) + : : : : : : :- * Project (79) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (78) + : : : : : : : :- * Project (73) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (72) + : : : : : : : : :- * Project (67) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (66) + : : : : : : : : : :- * Project (64) + : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (63) + : : : : : : : : : : :- * Project (58) + : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (57) + : : : : : : : : : : : :- * Project (55) + : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : : : : : : : : : : :- * Project (49) + : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (48) + : : : : : : : : : : : : : :- * Project (43) + : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (42) + : : : : : : : : : : : : : : :- * Project (37) + : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (36) + : : : : : : : : : : : : : : : :- * Project (34) + : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (33) + : : : : : : : : : : : : : : : : :- * ColumnarToRow (11) + : : : : : : : : : : : : : : : : : +- CometSort (10) + : : : : : : : : : : : : : : : : : +- CometColumnarExchange (9) + : : : : : : : : : : : : : : : : : +- CometProject (8) + : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) + : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) + : : : : : : : : : : : : : : : : : : +- CometFilter (2) + : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : : : : : : : : : : : : +- CometProject (6) + : : : : : : : : : : : : : : : : : +- CometFilter (5) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : : : : : : : : : : : : : : : +- * Sort (32) + : : : : : : : : : : : : : : : : +- * Project (31) + : : : : : : : : : : : : : : : : +- * Filter (30) + : : : : : : : : : : : : : : : : +- * HashAggregate (29) + : : : : : : : : : : : : : : : : +- * ColumnarToRow (28) + : : : : : : : : : : : : : : : : +- CometColumnarExchange (27) + : : : : : : : : : : : : : : : : +- RowToColumnar (26) + : : : : : : : : : : : : : : : : +- * HashAggregate (25) + : : : : : : : : : : : : : : : : +- * ColumnarToRow (24) + : : : : : : : : : : : : : : : : +- CometProject (23) + : : : : : : : : : : : : : : : : +- CometSortMergeJoin (22) + : : : : : : : : : : : : : : : : :- CometSort (16) + : : : : : : : : : : : : : : : : : +- CometColumnarExchange (15) + : : : : : : : : : : : : : : : : : +- CometProject (14) + : : : : : : : : : : : : : : : : : +- CometFilter (13) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (12) + : : : : : : : : : : : : : : : : +- CometSort (21) + : : : : : : : : : : : : : : : : +- CometColumnarExchange (20) + : : : : : : : : : : : : : : : : +- CometProject (19) + : : : : : : : : : : : : : : : : +- CometFilter (18) + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (17) + : : : : : : : : : : : : : : : +- ReusedExchange (35) + : : : : : : : : : : : : : : +- BroadcastExchange (41) + : : : : : : : : : : : : : : +- * ColumnarToRow (40) + : : : : : : : : : : : : : : +- CometFilter (39) + : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (38) + : : : : : : : : : : : : : +- BroadcastExchange (47) + : : : : : : : : : : : : : +- * ColumnarToRow (46) + : : : : : : : : : : : : : +- CometFilter (45) + : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (44) + : : : : : : : : : : : : +- BroadcastExchange (53) + : : : : : : : : : : : : +- * ColumnarToRow (52) + : : : : : : : : : : : : +- CometFilter (51) + : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (50) + : : : : : : : : : : : +- ReusedExchange (56) + : : : : : : : : : : +- BroadcastExchange (62) + : : : : : : : : : : +- * ColumnarToRow (61) + : : : : : : : : : : +- CometFilter (60) + : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (59) + : : : : : : : : : +- ReusedExchange (65) + : : : : : : : : +- BroadcastExchange (71) + : : : : : : : : +- * ColumnarToRow (70) + : : : : : : : : +- CometFilter (69) + : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (68) + : : : : : : : +- BroadcastExchange (77) + : : : : : : : +- * ColumnarToRow (76) + : : : : : : : +- CometFilter (75) + : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (74) + : : : : : : +- ReusedExchange (80) + : : : : : +- BroadcastExchange (86) + : : : : : +- * ColumnarToRow (85) + : : : : : +- CometFilter (84) + : : : : : +- CometScan parquet spark_catalog.default.customer_address (83) + : : : : +- ReusedExchange (89) + : : : +- BroadcastExchange (95) + : : : +- * ColumnarToRow (94) + : : : +- CometFilter (93) + : : : +- CometScan parquet spark_catalog.default.income_band (92) + : : +- ReusedExchange (98) + : +- BroadcastExchange (105) + : +- * ColumnarToRow (104) + : +- CometProject (103) + : +- CometFilter (102) + : +- CometScan parquet spark_catalog.default.item (101) + +- * ColumnarToRow (183) + +- CometSort (182) + +- CometColumnarExchange (181) + +- RowToColumnar (180) + +- * HashAggregate (179) + +- * HashAggregate (178) + +- * Project (177) + +- * BroadcastHashJoin Inner BuildRight (176) + :- * Project (174) + : +- * BroadcastHashJoin Inner BuildRight (173) + : :- * Project (171) + : : +- * BroadcastHashJoin Inner BuildRight (170) + : : :- * Project (168) + : : : +- * BroadcastHashJoin Inner BuildRight (167) + : : : :- * Project (165) + : : : : +- * BroadcastHashJoin Inner BuildRight (164) + : : : : :- * Project (162) + : : : : : +- * BroadcastHashJoin Inner BuildRight (161) + : : : : : :- * Project (159) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (158) + : : : : : : :- * Project (156) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (155) + : : : : : : : :- * Project (153) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (152) + : : : : : : : : :- * Project (150) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (149) + : : : : : : : : : :- * Project (147) + : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (146) + : : : : : : : : : : :- * Project (144) + : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (143) + : : : : : : : : : : : :- * Project (141) + : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (140) + : : : : : : : : : : : : :- * Project (138) + : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (137) + : : : : : : : : : : : : : :- * Project (135) + : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (134) + : : : : : : : : : : : : : : :- * Project (132) + : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (131) + : : : : : : : : : : : : : : : :- * ColumnarToRow (124) + : : : : : : : : : : : : : : : : +- CometSort (123) + : : : : : : : : : : : : : : : : +- CometColumnarExchange (122) + : : : : : : : : : : : : : : : : +- CometProject (121) + : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (120) + : : : : : : : : : : : : : : : : :- CometBroadcastExchange (116) + : : : : : : : : : : : : : : : : : +- CometFilter (115) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (114) + : : : : : : : : : : : : : : : : +- CometProject (119) + : : : : : : : : : : : : : : : : +- CometFilter (118) + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (117) + : : : : : : : : : : : : : : : +- * Sort (130) + : : : : : : : : : : : : : : : +- * Project (129) + : : : : : : : : : : : : : : : +- * Filter (128) + : : : : : : : : : : : : : : : +- * HashAggregate (127) + : : : : : : : : : : : : : : : +- * ColumnarToRow (126) + : : : : : : : : : : : : : : : +- ReusedExchange (125) + : : : : : : : : : : : : : : +- ReusedExchange (133) + : : : : : : : : : : : : : +- ReusedExchange (136) + : : : : : : : : : : : : +- ReusedExchange (139) + : : : : : : : : : : : +- ReusedExchange (142) + : : : : : : : : : : +- ReusedExchange (145) + : : : : : : : : : +- ReusedExchange (148) + : : : : : : : : +- ReusedExchange (151) + : : : : : : : +- ReusedExchange (154) + : : : : : : +- ReusedExchange (157) + : : : : : +- ReusedExchange (160) + : : : : +- ReusedExchange (163) + : : : +- ReusedExchange (166) + : : +- ReusedExchange (169) + : +- ReusedExchange (172) + +- ReusedExchange (175) (1) Scan parquet spark_catalog.default.store_sales @@ -222,16 +230,16 @@ Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number# 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] Arguments: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -(9) ColumnarToRow [codegen id : 1] +(9) CometColumnarExchange Input [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(10) Exchange +(10) CometSort Input [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1 ASC NULLS FIRST] -(11) Sort [codegen id : 2] +(11) ColumnarToRow [codegen id : 1] Input [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 (12) Scan parquet spark_catalog.default.catalog_sales Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] @@ -248,450 +256,458 @@ Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -(15) ColumnarToRow [codegen id : 3] +(15) CometColumnarExchange Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] +Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) Exchange +(16) CometSort Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST] -(17) Sort [codegen id : 4] -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST], false, 0 - -(18) Scan parquet spark_catalog.default.catalog_returns +(17) Scan parquet spark_catalog.default.catalog_returns Output [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(19) CometFilter +(18) CometFilter Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) -(20) CometProject +(19) CometProject Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -(21) ColumnarToRow [codegen id : 5] +(20) CometColumnarExchange Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) Exchange +(21) CometSort Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST] -(23) Sort [codegen id : 6] -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST], false, 0 +(22) CometSortMergeJoin +Left output [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] +Right output [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number#22], Inner -(24) SortMergeJoin [codegen id : 7] -Left keys [2]: [cs_item_sk#17, cs_order_number#18] -Right keys [2]: [cr_item_sk#21, cr_order_number#22] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 7] -Output [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +(23) CometProject Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +Arguments: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] + +(24) ColumnarToRow [codegen id : 2] +Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -(26) HashAggregate [codegen id : 7] +(25) HashAggregate [codegen id : 2] Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Keys [1]: [cs_item_sk#17] Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] Aggregate Attributes [3]: [sum#27, sum#28, isEmpty#29] Results [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] -(27) Exchange +(26) RowToColumnar +Input [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] + +(27) CometColumnarExchange Input [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(28) HashAggregate [codegen id : 8] +(28) ColumnarToRow [codegen id : 3] +Input [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] + +(29) HashAggregate [codegen id : 3] Input [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] Keys [1]: [cs_item_sk#17] Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#19))#33, sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))#34] Results [3]: [cs_item_sk#17, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#19))#33,17,2) AS sale#35, sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))#34 AS refund#36] -(29) Filter [codegen id : 8] +(30) Filter [codegen id : 3] Input [3]: [cs_item_sk#17, sale#35, refund#36] Condition : ((isnotnull(sale#35) AND isnotnull(refund#36)) AND (cast(sale#35 as decimal(21,2)) > (2 * refund#36))) -(30) Project [codegen id : 8] +(31) Project [codegen id : 3] Output [1]: [cs_item_sk#17] Input [3]: [cs_item_sk#17, sale#35, refund#36] -(31) Sort [codegen id : 8] +(32) Sort [codegen id : 3] Input [1]: [cs_item_sk#17] Arguments: [cs_item_sk#17 ASC NULLS FIRST], false, 0 -(32) SortMergeJoin [codegen id : 24] +(33) SortMergeJoin [codegen id : 19] Left keys [1]: [ss_item_sk#1] Right keys [1]: [cs_item_sk#17] Join type: Inner Join condition: None -(33) Project [codegen id : 24] +(34) Project [codegen id : 19] Output [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] Input [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#17] -(34) ReusedExchange [Reuses operator id: 185] +(35) ReusedExchange [Reuses operator id: 193] Output [2]: [d_date_sk#37, d_year#38] -(35) BroadcastHashJoin [codegen id : 24] +(36) BroadcastHashJoin [codegen id : 19] Left keys [1]: [ss_sold_date_sk#12] Right keys [1]: [d_date_sk#37] Join type: Inner Join condition: None -(36) Project [codegen id : 24] +(37) Project [codegen id : 19] Output [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38] Input [13]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#37, d_year#38] -(37) Scan parquet spark_catalog.default.store +(38) Scan parquet spark_catalog.default.store Output [3]: [s_store_sk#39, s_store_name#40, s_zip#41] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_zip)] ReadSchema: struct -(38) CometFilter +(39) CometFilter Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] Condition : ((isnotnull(s_store_sk#39) AND isnotnull(s_store_name#40)) AND isnotnull(s_zip#41)) -(39) ColumnarToRow [codegen id : 10] +(40) ColumnarToRow [codegen id : 5] Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] -(40) BroadcastExchange +(41) BroadcastExchange Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(41) BroadcastHashJoin [codegen id : 24] +(42) BroadcastHashJoin [codegen id : 19] Left keys [1]: [ss_store_sk#6] Right keys [1]: [s_store_sk#39] Join type: Inner Join condition: None -(42) Project [codegen id : 24] +(43) Project [codegen id : 19] Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41] 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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_sk#39, s_store_name#40, s_zip#41] -(43) Scan parquet spark_catalog.default.customer +(44) Scan parquet spark_catalog.default.customer Output [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(44) CometFilter +(45) CometFilter Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] Condition : (((((isnotnull(c_customer_sk#42) AND isnotnull(c_first_sales_date_sk#47)) AND isnotnull(c_first_shipto_date_sk#46)) AND isnotnull(c_current_cdemo_sk#43)) AND isnotnull(c_current_hdemo_sk#44)) AND isnotnull(c_current_addr_sk#45)) -(45) ColumnarToRow [codegen id : 11] +(46) ColumnarToRow [codegen id : 6] Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] -(46) BroadcastExchange +(47) BroadcastExchange Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(47) BroadcastHashJoin [codegen id : 24] +(48) BroadcastHashJoin [codegen id : 19] Left keys [1]: [ss_customer_sk#2] Right keys [1]: [c_customer_sk#42] Join type: Inner Join condition: None -(48) Project [codegen id : 24] +(49) Project [codegen id : 19] Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] -(49) Scan parquet spark_catalog.default.date_dim +(50) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#48, d_year#49] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct -(50) CometFilter +(51) CometFilter Input [2]: [d_date_sk#48, d_year#49] Condition : isnotnull(d_date_sk#48) -(51) ColumnarToRow [codegen id : 12] +(52) ColumnarToRow [codegen id : 7] Input [2]: [d_date_sk#48, d_year#49] -(52) BroadcastExchange +(53) BroadcastExchange Input [2]: [d_date_sk#48, d_year#49] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(53) BroadcastHashJoin [codegen id : 24] +(54) BroadcastHashJoin [codegen id : 19] Left keys [1]: [c_first_sales_date_sk#47] Right keys [1]: [d_date_sk#48] Join type: Inner Join condition: None -(54) Project [codegen id : 24] +(55) Project [codegen id : 19] Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, d_year#49] Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47, d_date_sk#48, d_year#49] -(55) ReusedExchange [Reuses operator id: 52] +(56) ReusedExchange [Reuses operator id: 53] Output [2]: [d_date_sk#50, d_year#51] -(56) BroadcastHashJoin [codegen id : 24] +(57) BroadcastHashJoin [codegen id : 19] Left keys [1]: [c_first_shipto_date_sk#46] Right keys [1]: [d_date_sk#50] Join type: Inner Join condition: None -(57) Project [codegen id : 24] +(58) Project [codegen id : 19] Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, d_year#49, d_date_sk#50, d_year#51] -(58) Scan parquet spark_catalog.default.customer_demographics +(59) Scan parquet spark_catalog.default.customer_demographics Output [2]: [cd_demo_sk#52, cd_marital_status#53] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status)] ReadSchema: struct -(59) CometFilter +(60) CometFilter Input [2]: [cd_demo_sk#52, cd_marital_status#53] Condition : (isnotnull(cd_demo_sk#52) AND isnotnull(cd_marital_status#53)) -(60) ColumnarToRow [codegen id : 14] +(61) ColumnarToRow [codegen id : 9] Input [2]: [cd_demo_sk#52, cd_marital_status#53] -(61) BroadcastExchange +(62) BroadcastExchange Input [2]: [cd_demo_sk#52, cd_marital_status#53] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(62) BroadcastHashJoin [codegen id : 24] +(63) BroadcastHashJoin [codegen id : 19] Left keys [1]: [ss_cdemo_sk#3] Right keys [1]: [cd_demo_sk#52] Join type: Inner Join condition: None -(63) Project [codegen id : 24] +(64) Project [codegen id : 19] Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_marital_status#53] Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_demo_sk#52, cd_marital_status#53] -(64) ReusedExchange [Reuses operator id: 61] +(65) ReusedExchange [Reuses operator id: 62] Output [2]: [cd_demo_sk#54, cd_marital_status#55] -(65) BroadcastHashJoin [codegen id : 24] +(66) BroadcastHashJoin [codegen id : 19] Left keys [1]: [c_current_cdemo_sk#43] Right keys [1]: [cd_demo_sk#54] Join type: Inner Join condition: NOT (cd_marital_status#53 = cd_marital_status#55) -(66) Project [codegen id : 24] +(67) Project [codegen id : 19] Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_marital_status#53, cd_demo_sk#54, cd_marital_status#55] -(67) Scan parquet spark_catalog.default.promotion +(68) Scan parquet spark_catalog.default.promotion Output [1]: [p_promo_sk#56] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_promo_sk)] ReadSchema: struct -(68) CometFilter +(69) CometFilter Input [1]: [p_promo_sk#56] Condition : isnotnull(p_promo_sk#56) -(69) ColumnarToRow [codegen id : 16] +(70) ColumnarToRow [codegen id : 11] Input [1]: [p_promo_sk#56] -(70) BroadcastExchange +(71) BroadcastExchange Input [1]: [p_promo_sk#56] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(71) BroadcastHashJoin [codegen id : 24] +(72) BroadcastHashJoin [codegen id : 19] Left keys [1]: [ss_promo_sk#7] Right keys [1]: [p_promo_sk#56] Join type: Inner Join condition: None -(72) Project [codegen id : 24] +(73) Project [codegen id : 19] Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, p_promo_sk#56] -(73) Scan parquet spark_catalog.default.household_demographics +(74) Scan parquet spark_catalog.default.household_demographics Output [2]: [hd_demo_sk#57, hd_income_band_sk#58] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] ReadSchema: struct -(74) CometFilter +(75) CometFilter Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] Condition : (isnotnull(hd_demo_sk#57) AND isnotnull(hd_income_band_sk#58)) -(75) ColumnarToRow [codegen id : 17] +(76) ColumnarToRow [codegen id : 12] Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] -(76) BroadcastExchange +(77) BroadcastExchange Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -(77) BroadcastHashJoin [codegen id : 24] +(78) BroadcastHashJoin [codegen id : 19] Left keys [1]: [ss_hdemo_sk#4] Right keys [1]: [hd_demo_sk#57] Join type: Inner Join condition: None -(78) Project [codegen id : 24] +(79) Project [codegen id : 19] Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58] Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_demo_sk#57, hd_income_band_sk#58] -(79) ReusedExchange [Reuses operator id: 76] +(80) ReusedExchange [Reuses operator id: 77] Output [2]: [hd_demo_sk#59, hd_income_band_sk#60] -(80) BroadcastHashJoin [codegen id : 24] +(81) BroadcastHashJoin [codegen id : 19] Left keys [1]: [c_current_hdemo_sk#44] Right keys [1]: [hd_demo_sk#59] Join type: Inner Join condition: None -(81) Project [codegen id : 24] +(82) Project [codegen id : 19] Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60] Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_demo_sk#59, hd_income_band_sk#60] -(82) Scan parquet spark_catalog.default.customer_address +(83) Scan parquet spark_catalog.default.customer_address Output [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(83) CometFilter +(84) CometFilter Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] Condition : isnotnull(ca_address_sk#61) -(84) ColumnarToRow [codegen id : 19] +(85) ColumnarToRow [codegen id : 14] Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -(85) BroadcastExchange +(86) BroadcastExchange Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=11] -(86) BroadcastHashJoin [codegen id : 24] +(87) BroadcastHashJoin [codegen id : 19] Left keys [1]: [ss_addr_sk#5] Right keys [1]: [ca_address_sk#61] Join type: Inner Join condition: None -(87) Project [codegen id : 24] +(88) Project [codegen id : 19] Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -(88) ReusedExchange [Reuses operator id: 85] +(89) ReusedExchange [Reuses operator id: 86] Output [5]: [ca_address_sk#66, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] -(89) BroadcastHashJoin [codegen id : 24] +(90) BroadcastHashJoin [codegen id : 19] Left keys [1]: [c_current_addr_sk#45] Right keys [1]: [ca_address_sk#66] Join type: Inner Join condition: None -(90) Project [codegen id : 24] +(91) Project [codegen id : 19] Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_address_sk#66, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] -(91) Scan parquet spark_catalog.default.income_band +(92) Scan parquet spark_catalog.default.income_band Output [1]: [ib_income_band_sk#71] Batched: true Location [not included in comparison]/{warehouse_dir}/income_band] PushedFilters: [IsNotNull(ib_income_band_sk)] ReadSchema: struct -(92) CometFilter +(93) CometFilter Input [1]: [ib_income_band_sk#71] Condition : isnotnull(ib_income_band_sk#71) -(93) ColumnarToRow [codegen id : 21] +(94) ColumnarToRow [codegen id : 16] Input [1]: [ib_income_band_sk#71] -(94) BroadcastExchange +(95) BroadcastExchange Input [1]: [ib_income_band_sk#71] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -(95) BroadcastHashJoin [codegen id : 24] +(96) BroadcastHashJoin [codegen id : 19] Left keys [1]: [hd_income_band_sk#58] Right keys [1]: [ib_income_band_sk#71] Join type: Inner Join condition: None -(96) Project [codegen id : 24] +(97) Project [codegen id : 19] Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, ib_income_band_sk#71] -(97) ReusedExchange [Reuses operator id: 94] +(98) ReusedExchange [Reuses operator id: 95] Output [1]: [ib_income_band_sk#72] -(98) BroadcastHashJoin [codegen id : 24] +(99) BroadcastHashJoin [codegen id : 19] Left keys [1]: [hd_income_band_sk#60] Right keys [1]: [ib_income_band_sk#72] Join type: Inner Join condition: None -(99) Project [codegen id : 24] +(100) Project [codegen id : 19] Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, ib_income_band_sk#72] -(100) Scan parquet spark_catalog.default.item +(101) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), In(i_color, [burlywood ,floral ,indian ,medium ,purple ,spring ]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] ReadSchema: struct -(101) CometFilter +(102) CometFilter Input [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] Condition : ((((((isnotnull(i_current_price#74) AND i_color#75 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#74 >= 64.00)) AND (i_current_price#74 <= 74.00)) AND (i_current_price#74 >= 65.00)) AND (i_current_price#74 <= 79.00)) AND isnotnull(i_item_sk#73)) -(102) CometProject +(103) CometProject Input [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] Arguments: [i_item_sk#73, i_product_name#76], [i_item_sk#73, i_product_name#76] -(103) ColumnarToRow [codegen id : 23] +(104) ColumnarToRow [codegen id : 18] Input [2]: [i_item_sk#73, i_product_name#76] -(104) BroadcastExchange +(105) BroadcastExchange Input [2]: [i_item_sk#73, i_product_name#76] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -(105) BroadcastHashJoin [codegen id : 24] +(106) BroadcastHashJoin [codegen id : 19] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#73] Join type: Inner Join condition: None -(106) Project [codegen id : 24] +(107) Project [codegen id : 19] Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, d_year#49, d_year#51, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] -(107) HashAggregate [codegen id : 24] +(108) HashAggregate [codegen id : 19] Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, d_year#49, d_year#51, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] Keys [15]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51] Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] Aggregate Attributes [4]: [count#77, sum#78, sum#79, sum#80] Results [19]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51, count#81, sum#82, sum#83, sum#84] -(108) HashAggregate [codegen id : 24] +(109) HashAggregate [codegen id : 19] Input [19]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51, count#81, sum#82, sum#83, sum#84] Keys [15]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51] Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] Aggregate Attributes [4]: [count(1)#85, sum(UnscaledValue(ss_wholesale_cost#9))#86, sum(UnscaledValue(ss_list_price#10))#87, sum(UnscaledValue(ss_coupon_amt#11))#88] Results [17]: [i_product_name#76 AS product_name#89, i_item_sk#73 AS item_sk#90, s_store_name#40 AS store_name#91, s_zip#41 AS store_zip#92, ca_street_number#62 AS b_street_number#93, ca_street_name#63 AS b_streen_name#94, ca_city#64 AS b_city#95, ca_zip#65 AS b_zip#96, ca_street_number#67 AS c_street_number#97, ca_street_name#68 AS c_street_name#98, ca_city#69 AS c_city#99, ca_zip#70 AS c_zip#100, d_year#38 AS syear#101, count(1)#85 AS cnt#102, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#86,17,2) AS s1#103, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#87,17,2) AS s2#104, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#88,17,2) AS s3#105] -(109) Exchange +(110) RowToColumnar +Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] + +(111) CometColumnarExchange Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] -Arguments: hashpartitioning(item_sk#90, store_name#91, store_zip#92, 5), ENSURE_REQUIREMENTS, [plan_id=14] +Arguments: hashpartitioning(item_sk#90, store_name#91, store_zip#92, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] -(110) Sort [codegen id : 25] +(112) CometSort Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] -Arguments: [item_sk#90 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, store_zip#92 ASC NULLS FIRST], false, 0 +Arguments: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105], [item_sk#90 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, store_zip#92 ASC NULLS FIRST] -(111) Scan parquet spark_catalog.default.store_sales +(113) ColumnarToRow [codegen id : 20] +Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] + +(114) Scan parquet spark_catalog.default.store_sales 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] Batched: true Location: InMemoryFileIndex [] @@ -699,365 +715,380 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#117), dynamicpruningexpression(ss_s PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct -(112) CometFilter +(115) CometFilter Input [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] Condition : (((((((isnotnull(ss_item_sk#106) AND isnotnull(ss_ticket_number#113)) AND isnotnull(ss_store_sk#111)) AND isnotnull(ss_customer_sk#107)) AND isnotnull(ss_cdemo_sk#108)) AND isnotnull(ss_promo_sk#112)) AND isnotnull(ss_hdemo_sk#109)) AND isnotnull(ss_addr_sk#110)) -(113) CometBroadcastExchange +(116) CometBroadcastExchange Input [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] Arguments: [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] -(114) Scan parquet spark_catalog.default.store_returns +(117) Scan parquet spark_catalog.default.store_returns Output [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] ReadSchema: struct -(115) CometFilter +(118) CometFilter Input [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] Condition : (isnotnull(sr_item_sk#119) AND isnotnull(sr_ticket_number#120)) -(116) CometProject +(119) CometProject Input [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] Arguments: [sr_item_sk#119, sr_ticket_number#120], [sr_item_sk#119, sr_ticket_number#120] -(117) CometBroadcastHashJoin +(120) 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, BuildLeft -(118) CometProject +(121) 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] Arguments: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117], [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -(119) ColumnarToRow [codegen id : 26] +(122) CometColumnarExchange Input [11]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Arguments: hashpartitioning(ss_item_sk#106, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] -(120) Exchange +(123) CometSort Input [11]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -Arguments: hashpartitioning(ss_item_sk#106, 5), ENSURE_REQUIREMENTS, [plan_id=15] +Arguments: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117], [ss_item_sk#106 ASC NULLS FIRST] -(121) Sort [codegen id : 27] +(124) ColumnarToRow [codegen id : 21] Input [11]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -Arguments: [ss_item_sk#106 ASC NULLS FIRST], false, 0 -(122) ReusedExchange [Reuses operator id: 27] +(125) ReusedExchange [Reuses operator id: 27] Output [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] -(123) HashAggregate [codegen id : 33] +(126) ColumnarToRow [codegen id : 23] +Input [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] + +(127) HashAggregate [codegen id : 23] Input [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] Keys [1]: [cs_item_sk#122] Functions [2]: [sum(UnscaledValue(cs_ext_list_price#126)), sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#126))#33, sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))#34] Results [3]: [cs_item_sk#122, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#126))#33,17,2) AS sale#130, sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))#34 AS refund#131] -(124) Filter [codegen id : 33] +(128) Filter [codegen id : 23] Input [3]: [cs_item_sk#122, sale#130, refund#131] Condition : ((isnotnull(sale#130) AND isnotnull(refund#131)) AND (cast(sale#130 as decimal(21,2)) > (2 * refund#131))) -(125) Project [codegen id : 33] +(129) Project [codegen id : 23] Output [1]: [cs_item_sk#122] Input [3]: [cs_item_sk#122, sale#130, refund#131] -(126) Sort [codegen id : 33] +(130) Sort [codegen id : 23] Input [1]: [cs_item_sk#122] Arguments: [cs_item_sk#122 ASC NULLS FIRST], false, 0 -(127) SortMergeJoin [codegen id : 49] +(131) SortMergeJoin [codegen id : 39] Left keys [1]: [ss_item_sk#106] Right keys [1]: [cs_item_sk#122] Join type: Inner Join condition: None -(128) Project [codegen id : 49] +(132) Project [codegen id : 39] Output [11]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] Input [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, cs_item_sk#122] -(129) ReusedExchange [Reuses operator id: 189] +(133) ReusedExchange [Reuses operator id: 197] Output [2]: [d_date_sk#132, d_year#133] -(130) BroadcastHashJoin [codegen id : 49] +(134) BroadcastHashJoin [codegen id : 39] Left keys [1]: [ss_sold_date_sk#117] Right keys [1]: [d_date_sk#132] Join type: Inner Join condition: None -(131) Project [codegen id : 49] +(135) Project [codegen id : 39] Output [11]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133] Input [13]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, d_date_sk#132, d_year#133] -(132) ReusedExchange [Reuses operator id: 40] +(136) ReusedExchange [Reuses operator id: 41] Output [3]: [s_store_sk#134, s_store_name#135, s_zip#136] -(133) BroadcastHashJoin [codegen id : 49] +(137) BroadcastHashJoin [codegen id : 39] Left keys [1]: [ss_store_sk#111] Right keys [1]: [s_store_sk#134] Join type: Inner Join condition: None -(134) Project [codegen id : 49] +(138) Project [codegen id : 39] Output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136] 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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_sk#134, s_store_name#135, s_zip#136] -(135) ReusedExchange [Reuses operator id: 46] +(139) ReusedExchange [Reuses operator id: 47] Output [6]: [c_customer_sk#137, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, c_first_shipto_date_sk#141, c_first_sales_date_sk#142] -(136) BroadcastHashJoin [codegen id : 49] +(140) BroadcastHashJoin [codegen id : 39] Left keys [1]: [ss_customer_sk#107] Right keys [1]: [c_customer_sk#137] Join type: Inner Join condition: None -(137) Project [codegen id : 49] +(141) Project [codegen id : 39] Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, c_first_shipto_date_sk#141, c_first_sales_date_sk#142] Input [18]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_customer_sk#137, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, c_first_shipto_date_sk#141, c_first_sales_date_sk#142] -(138) ReusedExchange [Reuses operator id: 52] +(142) ReusedExchange [Reuses operator id: 53] Output [2]: [d_date_sk#143, d_year#144] -(139) BroadcastHashJoin [codegen id : 49] +(143) BroadcastHashJoin [codegen id : 39] Left keys [1]: [c_first_sales_date_sk#142] Right keys [1]: [d_date_sk#143] Join type: Inner Join condition: None -(140) Project [codegen id : 49] +(144) Project [codegen id : 39] Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, c_first_shipto_date_sk#141, d_year#144] Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, c_first_shipto_date_sk#141, c_first_sales_date_sk#142, d_date_sk#143, d_year#144] -(141) ReusedExchange [Reuses operator id: 52] +(145) ReusedExchange [Reuses operator id: 53] Output [2]: [d_date_sk#145, d_year#146] -(142) BroadcastHashJoin [codegen id : 49] +(146) BroadcastHashJoin [codegen id : 39] Left keys [1]: [c_first_shipto_date_sk#141] Right keys [1]: [d_date_sk#145] Join type: Inner Join condition: None -(143) Project [codegen id : 49] +(147) Project [codegen id : 39] Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146] Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, c_first_shipto_date_sk#141, d_year#144, d_date_sk#145, d_year#146] -(144) ReusedExchange [Reuses operator id: 61] +(148) ReusedExchange [Reuses operator id: 62] Output [2]: [cd_demo_sk#147, cd_marital_status#148] -(145) BroadcastHashJoin [codegen id : 49] +(149) BroadcastHashJoin [codegen id : 39] Left keys [1]: [ss_cdemo_sk#108] Right keys [1]: [cd_demo_sk#147] Join type: Inner Join condition: None -(146) Project [codegen id : 49] +(150) Project [codegen id : 39] Output [16]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146, cd_marital_status#148] Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146, cd_demo_sk#147, cd_marital_status#148] -(147) ReusedExchange [Reuses operator id: 61] +(151) ReusedExchange [Reuses operator id: 62] Output [2]: [cd_demo_sk#149, cd_marital_status#150] -(148) BroadcastHashJoin [codegen id : 49] +(152) BroadcastHashJoin [codegen id : 39] Left keys [1]: [c_current_cdemo_sk#138] Right keys [1]: [cd_demo_sk#149] Join type: Inner Join condition: NOT (cd_marital_status#148 = cd_marital_status#150) -(149) Project [codegen id : 49] +(153) Project [codegen id : 39] Output [14]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146] Input [18]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146, cd_marital_status#148, cd_demo_sk#149, cd_marital_status#150] -(150) ReusedExchange [Reuses operator id: 70] +(154) ReusedExchange [Reuses operator id: 71] Output [1]: [p_promo_sk#151] -(151) BroadcastHashJoin [codegen id : 49] +(155) BroadcastHashJoin [codegen id : 39] Left keys [1]: [ss_promo_sk#112] Right keys [1]: [p_promo_sk#151] Join type: Inner Join condition: None -(152) Project [codegen id : 49] +(156) Project [codegen id : 39] Output [13]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146] Input [15]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146, p_promo_sk#151] -(153) ReusedExchange [Reuses operator id: 76] +(157) ReusedExchange [Reuses operator id: 77] Output [2]: [hd_demo_sk#152, hd_income_band_sk#153] -(154) BroadcastHashJoin [codegen id : 49] +(158) BroadcastHashJoin [codegen id : 39] Left keys [1]: [ss_hdemo_sk#109] Right keys [1]: [hd_demo_sk#152] Join type: Inner Join condition: None -(155) Project [codegen id : 49] +(159) Project [codegen id : 39] Output [13]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146, hd_income_band_sk#153] Input [15]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146, hd_demo_sk#152, hd_income_band_sk#153] -(156) ReusedExchange [Reuses operator id: 76] +(160) ReusedExchange [Reuses operator id: 77] Output [2]: [hd_demo_sk#154, hd_income_band_sk#155] -(157) BroadcastHashJoin [codegen id : 49] +(161) BroadcastHashJoin [codegen id : 39] Left keys [1]: [c_current_hdemo_sk#139] Right keys [1]: [hd_demo_sk#154] Join type: Inner Join condition: None -(158) Project [codegen id : 49] +(162) Project [codegen id : 39] Output [13]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_addr_sk#140, d_year#144, d_year#146, hd_income_band_sk#153, hd_income_band_sk#155] Input [15]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146, hd_income_band_sk#153, hd_demo_sk#154, hd_income_band_sk#155] -(159) ReusedExchange [Reuses operator id: 85] +(163) ReusedExchange [Reuses operator id: 86] Output [5]: [ca_address_sk#156, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160] -(160) BroadcastHashJoin [codegen id : 49] +(164) BroadcastHashJoin [codegen id : 39] Left keys [1]: [ss_addr_sk#110] Right keys [1]: [ca_address_sk#156] Join type: Inner Join condition: None -(161) Project [codegen id : 49] +(165) Project [codegen id : 39] Output [16]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_addr_sk#140, d_year#144, d_year#146, hd_income_band_sk#153, hd_income_band_sk#155, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160] Input [18]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_addr_sk#140, d_year#144, d_year#146, hd_income_band_sk#153, hd_income_band_sk#155, ca_address_sk#156, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160] -(162) ReusedExchange [Reuses operator id: 85] +(166) ReusedExchange [Reuses operator id: 86] Output [5]: [ca_address_sk#161, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165] -(163) BroadcastHashJoin [codegen id : 49] +(167) BroadcastHashJoin [codegen id : 39] Left keys [1]: [c_current_addr_sk#140] Right keys [1]: [ca_address_sk#161] Join type: Inner Join condition: None -(164) Project [codegen id : 49] +(168) Project [codegen id : 39] Output [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, d_year#144, d_year#146, hd_income_band_sk#153, hd_income_band_sk#155, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165] Input [21]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_addr_sk#140, d_year#144, d_year#146, hd_income_band_sk#153, hd_income_band_sk#155, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_address_sk#161, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165] -(165) ReusedExchange [Reuses operator id: 94] +(169) ReusedExchange [Reuses operator id: 95] Output [1]: [ib_income_band_sk#166] -(166) BroadcastHashJoin [codegen id : 49] +(170) BroadcastHashJoin [codegen id : 39] Left keys [1]: [hd_income_band_sk#153] Right keys [1]: [ib_income_band_sk#166] Join type: Inner Join condition: None -(167) Project [codegen id : 49] +(171) Project [codegen id : 39] Output [18]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, d_year#144, d_year#146, hd_income_band_sk#155, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165] Input [20]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, d_year#144, d_year#146, hd_income_band_sk#153, hd_income_band_sk#155, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, ib_income_band_sk#166] -(168) ReusedExchange [Reuses operator id: 94] +(172) ReusedExchange [Reuses operator id: 95] Output [1]: [ib_income_band_sk#167] -(169) BroadcastHashJoin [codegen id : 49] +(173) BroadcastHashJoin [codegen id : 39] Left keys [1]: [hd_income_band_sk#155] Right keys [1]: [ib_income_band_sk#167] Join type: Inner Join condition: None -(170) Project [codegen id : 49] +(174) Project [codegen id : 39] Output [17]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, d_year#144, d_year#146, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165] Input [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, d_year#144, d_year#146, hd_income_band_sk#155, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, ib_income_band_sk#167] -(171) ReusedExchange [Reuses operator id: 104] +(175) ReusedExchange [Reuses operator id: 105] Output [2]: [i_item_sk#168, i_product_name#169] -(172) BroadcastHashJoin [codegen id : 49] +(176) BroadcastHashJoin [codegen id : 39] Left keys [1]: [ss_item_sk#106] Right keys [1]: [i_item_sk#168] Join type: Inner Join condition: None -(173) Project [codegen id : 49] +(177) Project [codegen id : 39] Output [18]: [ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, d_year#144, d_year#146, s_store_name#135, s_zip#136, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, i_item_sk#168, i_product_name#169] Input [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, d_year#144, d_year#146, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, i_item_sk#168, i_product_name#169] -(174) HashAggregate [codegen id : 49] +(178) HashAggregate [codegen id : 39] Input [18]: [ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, d_year#144, d_year#146, s_store_name#135, s_zip#136, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, i_item_sk#168, i_product_name#169] Keys [15]: [i_product_name#169, i_item_sk#168, s_store_name#135, s_zip#136, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, d_year#133, d_year#144, d_year#146] Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#114)), partial_sum(UnscaledValue(ss_list_price#115)), partial_sum(UnscaledValue(ss_coupon_amt#116))] Aggregate Attributes [4]: [count#77, sum#170, sum#171, sum#172] Results [19]: [i_product_name#169, i_item_sk#168, s_store_name#135, s_zip#136, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, d_year#133, d_year#144, d_year#146, count#81, sum#173, sum#174, sum#175] -(175) HashAggregate [codegen id : 49] +(179) HashAggregate [codegen id : 39] Input [19]: [i_product_name#169, i_item_sk#168, s_store_name#135, s_zip#136, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, d_year#133, d_year#144, d_year#146, count#81, sum#173, sum#174, sum#175] Keys [15]: [i_product_name#169, i_item_sk#168, s_store_name#135, s_zip#136, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, d_year#133, d_year#144, d_year#146] Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#114)), sum(UnscaledValue(ss_list_price#115)), sum(UnscaledValue(ss_coupon_amt#116))] Aggregate Attributes [4]: [count(1)#85, sum(UnscaledValue(ss_wholesale_cost#114))#86, sum(UnscaledValue(ss_list_price#115))#87, sum(UnscaledValue(ss_coupon_amt#116))#88] Results [8]: [i_item_sk#168 AS item_sk#176, s_store_name#135 AS store_name#177, s_zip#136 AS store_zip#178, d_year#133 AS syear#179, count(1)#85 AS cnt#180, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#114))#86,17,2) AS s1#181, MakeDecimal(sum(UnscaledValue(ss_list_price#115))#87,17,2) AS s2#182, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#116))#88,17,2) AS s3#183] -(176) Exchange +(180) RowToColumnar +Input [8]: [item_sk#176, store_name#177, store_zip#178, syear#179, cnt#180, s1#181, s2#182, s3#183] + +(181) CometColumnarExchange +Input [8]: [item_sk#176, store_name#177, store_zip#178, syear#179, cnt#180, s1#181, s2#182, s3#183] +Arguments: hashpartitioning(item_sk#176, store_name#177, store_zip#178, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] + +(182) CometSort Input [8]: [item_sk#176, store_name#177, store_zip#178, syear#179, cnt#180, s1#181, s2#182, s3#183] -Arguments: hashpartitioning(item_sk#176, store_name#177, store_zip#178, 5), ENSURE_REQUIREMENTS, [plan_id=16] +Arguments: [item_sk#176, store_name#177, store_zip#178, syear#179, cnt#180, s1#181, s2#182, s3#183], [item_sk#176 ASC NULLS FIRST, store_name#177 ASC NULLS FIRST, store_zip#178 ASC NULLS FIRST] -(177) Sort [codegen id : 50] +(183) ColumnarToRow [codegen id : 40] Input [8]: [item_sk#176, store_name#177, store_zip#178, syear#179, cnt#180, s1#181, s2#182, s3#183] -Arguments: [item_sk#176 ASC NULLS FIRST, store_name#177 ASC NULLS FIRST, store_zip#178 ASC NULLS FIRST], false, 0 -(178) SortMergeJoin [codegen id : 51] +(184) SortMergeJoin [codegen id : 41] Left keys [3]: [item_sk#90, store_name#91, store_zip#92] Right keys [3]: [item_sk#176, store_name#177, store_zip#178] Join type: Inner Join condition: (cnt#180 <= cnt#102) -(179) Project [codegen id : 51] +(185) Project [codegen id : 41] Output [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#181, s2#182, s3#183, syear#179, cnt#180] Input [25]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, item_sk#176, store_name#177, store_zip#178, syear#179, cnt#180, s1#181, s2#182, s3#183] -(180) Exchange +(186) RowToColumnar +Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#181, s2#182, s3#183, syear#179, cnt#180] + +(187) CometColumnarExchange +Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#181, s2#182, s3#183, syear#179, cnt#180] +Arguments: rangepartitioning(product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#180 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] + +(188) CometSort Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#181, s2#182, s3#183, syear#179, cnt#180] -Arguments: rangepartitioning(product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#180 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=17] +Arguments: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#181, s2#182, s3#183, syear#179, cnt#180], [product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#180 ASC NULLS FIRST] -(181) Sort [codegen id : 52] +(189) ColumnarToRow [codegen id : 42] Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#181, s2#182, s3#183, syear#179, cnt#180] -Arguments: [product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#180 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (185) -+- * ColumnarToRow (184) - +- CometFilter (183) - +- CometScan parquet spark_catalog.default.date_dim (182) +BroadcastExchange (193) ++- * ColumnarToRow (192) + +- CometFilter (191) + +- CometScan parquet spark_catalog.default.date_dim (190) -(182) Scan parquet spark_catalog.default.date_dim +(190) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#37, d_year#38] 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 -(183) CometFilter +(191) CometFilter Input [2]: [d_date_sk#37, d_year#38] Condition : ((isnotnull(d_year#38) AND (d_year#38 = 1999)) AND isnotnull(d_date_sk#37)) -(184) ColumnarToRow [codegen id : 1] +(192) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#37, d_year#38] -(185) BroadcastExchange +(193) BroadcastExchange Input [2]: [d_date_sk#37, d_year#38] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] -Subquery:2 Hosting operator id = 111 Hosting Expression = ss_sold_date_sk#117 IN dynamicpruning#118 -BroadcastExchange (189) -+- * ColumnarToRow (188) - +- CometFilter (187) - +- CometScan parquet spark_catalog.default.date_dim (186) +Subquery:2 Hosting operator id = 114 Hosting Expression = ss_sold_date_sk#117 IN dynamicpruning#118 +BroadcastExchange (197) ++- * ColumnarToRow (196) + +- CometFilter (195) + +- CometScan parquet spark_catalog.default.date_dim (194) -(186) Scan parquet spark_catalog.default.date_dim +(194) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#132, d_year#133] 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 -(187) CometFilter +(195) CometFilter Input [2]: [d_date_sk#132, d_year#133] Condition : ((isnotnull(d_year#133) AND (d_year#133 = 2000)) AND isnotnull(d_date_sk#132)) -(188) ColumnarToRow [codegen id : 1] +(196) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#132, d_year#133] -(189) BroadcastExchange +(197) BroadcastExchange Input [2]: [d_date_sk#132, d_year#133] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=19] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt index 3b5e4f19e..7209b28f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt @@ -1,281 +1,276 @@ -WholeStageCodegen (52) - Sort [product_name,store_name,cnt] +WholeStageCodegen (42) + ColumnarToRow InputAdapter - Exchange [product_name,store_name,cnt] #1 - WholeStageCodegen (51) - Project [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - SortMergeJoin [item_sk,store_name,store_zip,item_sk,store_name,store_zip,cnt,cnt] - InputAdapter - WholeStageCodegen (25) - Sort [item_sk,store_name,store_zip] - InputAdapter - Exchange [item_sk,store_name,store_zip] #2 - WholeStageCodegen (24) - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] - Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SortMergeJoin [ss_item_sk,cs_item_sk] - InputAdapter - WholeStageCodegen (2) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #3 + CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] + CometColumnarExchange [product_name,store_name,cnt] #1 + RowToColumnar + WholeStageCodegen (41) + Project [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] + SortMergeJoin [item_sk,store_name,store_zip,item_sk,store_name,store_zip,cnt,cnt] + InputAdapter + WholeStageCodegen (20) + ColumnarToRow + InputAdapter + CometSort [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3] + CometColumnarExchange [item_sk,store_name,store_zip] #2 + RowToColumnar + WholeStageCodegen (19) + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] + Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SortMergeJoin [ss_item_sk,cs_item_sk] + InputAdapter WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #4 - CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) + CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk] #3 + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #4 + CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + WholeStageCodegen (3) + Sort [cs_item_sk] + Project [cs_item_sk] + Filter [sale,refund] + HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [cs_item_sk] #6 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [sum,sum,isEmpty,sum,sum,isEmpty] ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometSortMergeJoin [cs_item_sk,cs_order_number,cs_ext_list_price,cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometSort [cs_item_sk,cs_order_number,cs_ext_list_price] + CometColumnarExchange [cs_item_sk,cs_order_number] #7 + CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometColumnarExchange [cr_item_sk,cr_order_number] #8 + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 InputAdapter - WholeStageCodegen (8) - Sort [cs_item_sk] - Project [cs_item_sk] - Filter [sale,refund] - HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] - InputAdapter - Exchange [cs_item_sk] #6 - WholeStageCodegen (7) - HashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [sum,sum,isEmpty,sum,sum,isEmpty] - Project [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] - InputAdapter - WholeStageCodegen (4) - Sort [cs_item_sk,cs_order_number] - InputAdapter - Exchange [cs_item_sk,cs_order_number] #7 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - InputAdapter - WholeStageCodegen (6) - Sort [cr_item_sk,cr_order_number] - InputAdapter - Exchange [cr_item_sk,cr_order_number] #8 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + BroadcastExchange #9 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk,s_store_name,s_zip] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] InputAdapter - ReusedExchange [d_date_sk,d_year] #5 + BroadcastExchange #10 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (10) + BroadcastExchange #11 + WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [s_store_sk,s_store_name,s_zip] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - BroadcastExchange #10 - WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + ReusedExchange [d_date_sk,d_year] #11 InputAdapter - BroadcastExchange #11 - WholeStageCodegen (12) + BroadcastExchange #12 + WholeStageCodegen (9) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometFilter [cd_demo_sk,cd_marital_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] InputAdapter - ReusedExchange [d_date_sk,d_year] #11 + ReusedExchange [cd_demo_sk,cd_marital_status] #12 InputAdapter - BroadcastExchange #12 - WholeStageCodegen (14) + BroadcastExchange #13 + WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk,cd_marital_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + CometFilter [p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk] InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 + BroadcastExchange #14 + WholeStageCodegen (12) + 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 #13 - WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometFilter [p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk] + ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 InputAdapter - BroadcastExchange #14 - WholeStageCodegen (17) + BroadcastExchange #15 + WholeStageCodegen (14) ColumnarToRow InputAdapter - CometFilter [hd_demo_sk,hd_income_band_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 InputAdapter - BroadcastExchange #15 - WholeStageCodegen (19) + BroadcastExchange #16 + WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometFilter [ib_income_band_sk] + CometScan parquet spark_catalog.default.income_band [ib_income_band_sk] InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 + ReusedExchange [ib_income_band_sk] #16 InputAdapter - BroadcastExchange #16 - WholeStageCodegen (21) + BroadcastExchange #17 + WholeStageCodegen (18) ColumnarToRow InputAdapter - CometFilter [ib_income_band_sk] - CometScan parquet spark_catalog.default.income_band [ib_income_band_sk] - InputAdapter - ReusedExchange [ib_income_band_sk] #16 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (23) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] - InputAdapter - WholeStageCodegen (50) - Sort [item_sk,store_name,store_zip] - InputAdapter - Exchange [item_sk,store_name,store_zip] #18 - WholeStageCodegen (49) - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] - Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SortMergeJoin [ss_item_sk,cs_item_sk] - InputAdapter - WholeStageCodegen (27) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #19 - WholeStageCodegen (26) + CometProject [i_item_sk,i_product_name] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] + InputAdapter + WholeStageCodegen (40) + ColumnarToRow + InputAdapter + CometSort [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] + CometColumnarExchange [item_sk,store_name,store_zip] #18 + RowToColumnar + WholeStageCodegen (39) + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] + Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SortMergeJoin [ss_item_sk,cs_item_sk] + InputAdapter + WholeStageCodegen (21) ColumnarToRow InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #20 - CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #21 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk] #19 + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #20 + CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #21 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + WholeStageCodegen (23) + Sort [cs_item_sk] + Project [cs_item_sk] + Filter [sale,refund] + HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] + ColumnarToRow + InputAdapter + ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #21 InputAdapter - WholeStageCodegen (33) - Sort [cs_item_sk] - Project [cs_item_sk] - Filter [sale,refund] - HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] - InputAdapter - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 + ReusedExchange [s_store_sk,s_store_name,s_zip] #9 InputAdapter - ReusedExchange [d_date_sk,d_year] #21 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #10 InputAdapter - ReusedExchange [s_store_sk,s_store_name,s_zip] #9 + ReusedExchange [d_date_sk,d_year] #11 InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #10 + ReusedExchange [d_date_sk,d_year] #11 InputAdapter - ReusedExchange [d_date_sk,d_year] #11 + ReusedExchange [cd_demo_sk,cd_marital_status] #12 InputAdapter - ReusedExchange [d_date_sk,d_year] #11 + ReusedExchange [cd_demo_sk,cd_marital_status] #12 InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 + ReusedExchange [p_promo_sk] #13 InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 + ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 InputAdapter - ReusedExchange [p_promo_sk] #13 + ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 + ReusedExchange [ib_income_band_sk] #16 InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 + ReusedExchange [ib_income_band_sk] #16 InputAdapter - ReusedExchange [ib_income_band_sk] #16 - InputAdapter - ReusedExchange [ib_income_band_sk] #16 - InputAdapter - ReusedExchange [i_item_sk,i_product_name] #17 + ReusedExchange [i_item_sk,i_product_name] #17 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 ea7975366..9b2797d93 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,46 +1,52 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * Project (41) - +- * BroadcastHashJoin Inner BuildRight (40) - :- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) +TakeOrderedAndProject (48) ++- * Project (47) + +- * BroadcastHashJoin Inner BuildRight (46) + :- * Project (27) + : +- * BroadcastHashJoin Inner BuildRight (26) + : :- * Project (21) + : : +- * BroadcastHashJoin Inner BuildRight (20) : : :- * ColumnarToRow (3) : : : +- CometFilter (2) : : : +- CometScan parquet spark_catalog.default.store (1) - : : +- 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) + : : +- BroadcastExchange (19) + : : +- * Filter (18) + : : +- * HashAggregate (17) + : : +- * ColumnarToRow (16) + : : +- CometColumnarExchange (15) + : : +- RowToColumnar (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 (25) + : +- * ColumnarToRow (24) + : +- CometFilter (23) + : +- CometScan parquet spark_catalog.default.item (22) + +- BroadcastExchange (45) + +- * Filter (44) + +- * HashAggregate (43) + +- * ColumnarToRow (42) + +- CometColumnarExchange (41) + +- RowToColumnar (40) + +- * HashAggregate (39) + +- * HashAggregate (38) + +- * ColumnarToRow (37) + +- CometColumnarExchange (36) + +- RowToColumnar (35) + +- * HashAggregate (34) + +- * ColumnarToRow (33) + +- CometProject (32) + +- CometBroadcastHashJoin (31) + :- CometFilter (29) + : +- CometScan parquet spark_catalog.default.store_sales (28) + +- ReusedExchange (30) (1) Scan parquet spark_catalog.default.store @@ -107,64 +113,70 @@ Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#5))] Aggregate Attributes [1]: [sum#10] Results [3]: [ss_store_sk#4, ss_item_sk#3, sum#11] -(14) Exchange +(14) RowToColumnar 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] -(15) HashAggregate [codegen id : 2] +(15) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(16) ColumnarToRow [codegen id : 2] +Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#11] + +(17) 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))#12] Results [3]: [ss_store_sk#4, ss_item_sk#3, MakeDecimal(sum(UnscaledValue(ss_sales_price#5))#12,17,2) AS revenue#13] -(16) Filter [codegen id : 2] +(18) Filter [codegen id : 2] Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#13] Condition : isnotnull(revenue#13) -(17) BroadcastExchange +(19) 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] -(18) BroadcastHashJoin [codegen id : 7] +(20) BroadcastHashJoin [codegen id : 7] Left keys [1]: [s_store_sk#1] Right keys [1]: [ss_store_sk#4] Join type: Inner Join condition: None -(19) Project [codegen id : 7] +(21) 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] -(20) Scan parquet spark_catalog.default.item +(22) 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 -(21) CometFilter +(23) 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) -(22) ColumnarToRow [codegen id : 3] +(24) 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] -(23) BroadcastExchange +(25) 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] -(24) BroadcastHashJoin [codegen id : 7] +(26) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ss_item_sk#3] Right keys [1]: [i_item_sk#14] Join type: Inner Join condition: None -(25) Project [codegen id : 7] +(27) 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] -(26) Scan parquet spark_catalog.default.store_sales +(28) 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 [] @@ -172,115 +184,127 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(27) CometFilter +(29) 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) -(28) ReusedExchange [Reuses operator id: 9] +(30) ReusedExchange [Reuses operator id: 9] Output [1]: [d_date_sk#24] -(29) CometBroadcastHashJoin +(31) 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 -(30) CometProject +(32) 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] +(33) ColumnarToRow [codegen id : 4] Input [3]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21] -(32) HashAggregate [codegen id : 4] +(34) 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 +(35) RowToColumnar +Input [3]: [ss_store_sk#20, ss_item_sk#19, sum#26] + +(36) CometColumnarExchange 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] +Arguments: hashpartitioning(ss_store_sk#20, ss_item_sk#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(34) HashAggregate [codegen id : 5] +(37) ColumnarToRow [codegen id : 5] +Input [3]: [ss_store_sk#20, ss_item_sk#19, sum#26] + +(38) 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] +(39) 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 +(40) RowToColumnar +Input [3]: [ss_store_sk#20, sum#31, count#32] + +(41) CometColumnarExchange +Input [3]: [ss_store_sk#20, sum#31, count#32] +Arguments: hashpartitioning(ss_store_sk#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(42) ColumnarToRow [codegen id : 6] 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] +(43) 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] +(44) Filter [codegen id : 6] Input [2]: [ss_store_sk#20, ave#34] Condition : isnotnull(ave#34) -(39) BroadcastExchange +(45) BroadcastExchange Input [2]: [ss_store_sk#20, ave#34] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(40) BroadcastHashJoin [codegen id : 7] +(46) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ss_store_sk#4] Right keys [1]: [ss_store_sk#20] Join type: Inner Join condition: (cast(revenue#13 as decimal(23,7)) <= (0.1 * ave#34)) -(41) Project [codegen id : 7] +(47) 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] -(42) TakeOrderedAndProject +(48) 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 (47) -+- * ColumnarToRow (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan parquet spark_catalog.default.date_dim (43) +BroadcastExchange (53) ++- * ColumnarToRow (52) + +- CometProject (51) + +- CometFilter (50) + +- CometScan parquet spark_catalog.default.date_dim (49) -(43) Scan parquet spark_catalog.default.date_dim +(49) 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 -(44) CometFilter +(50) 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)) -(45) CometProject +(51) CometProject Input [2]: [d_date_sk#8, d_month_seq#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(46) ColumnarToRow [codegen id : 1] +(52) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#8] -(47) BroadcastExchange +(53) 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 = 26 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 28 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 84d23af42..9480a4131 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 @@ -15,28 +15,30 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes 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 (1) - HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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 #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + ColumnarToRow + InputAdapter + CometColumnarExchange [ss_store_sk,ss_item_sk] #2 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_store_sk,ss_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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 #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) @@ -49,20 +51,24 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes WholeStageCodegen (6) Filter [ave] HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count] - InputAdapter - 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] #8 - WholeStageCodegen (4) - HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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 - ReusedExchange [d_date_sk] #4 + ColumnarToRow + InputAdapter + CometColumnarExchange [ss_store_sk] #7 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [ss_store_sk,ss_item_sk] #8 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_store_sk,ss_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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 + 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 ea11cbb74..2c6e3a331 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,55 +1,61 @@ == Physical Plan == -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) +TakeOrderedAndProject (57) ++- * HashAggregate (56) + +- * ColumnarToRow (55) + +- CometColumnarExchange (54) + +- RowToColumnar (53) + +- * HashAggregate (52) + +- Union (51) + :- * HashAggregate (30) + : +- * ColumnarToRow (29) + : +- CometColumnarExchange (28) + : +- RowToColumnar (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 (50) + +- * ColumnarToRow (49) + +- CometColumnarExchange (48) + +- RowToColumnar (47) + +- * HashAggregate (46) + +- * ColumnarToRow (45) + +- CometProject (44) + +- CometBroadcastHashJoin (43) + :- CometProject (41) + : +- CometBroadcastHashJoin (40) + : :- CometProject (38) + : : +- CometBroadcastHashJoin (37) + : : :- CometProject (35) + : : : +- CometBroadcastHashJoin (34) + : : : :- CometFilter (32) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (31) + : : : +- ReusedExchange (33) + : : +- ReusedExchange (36) + : +- ReusedExchange (39) + +- ReusedExchange (42) (1) Scan parquet spark_catalog.default.web_sales @@ -178,18 +184,24 @@ Functions [24]: [partial_sum(CASE WHEN (d_moy#18 = 1) THEN (ws_ext_sales_price#5 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] -(27) Exchange +(27) RowToColumnar 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=1] -(28) HashAggregate [codegen id : 2] +(28) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(29) ColumnarToRow [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] + +(30) 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] -(29) Scan parquet spark_catalog.default.catalog_sales +(31) 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 [] @@ -197,130 +209,142 @@ 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 -(30) CometFilter +(32) 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)) -(31) ReusedExchange [Reuses operator id: 5] +(33) 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] -(32) CometBroadcastHashJoin +(34) 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 -(33) CometProject +(35) 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] -(34) ReusedExchange [Reuses operator id: 10] +(36) ReusedExchange [Reuses operator id: 10] Output [3]: [d_date_sk#184, d_year#185, d_moy#186] -(35) CometBroadcastHashJoin +(37) 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 -(36) CometProject +(38) 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] -(37) ReusedExchange [Reuses operator id: 16] +(39) ReusedExchange [Reuses operator id: 16] Output [1]: [t_time_sk#187] -(38) CometBroadcastHashJoin +(40) 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 -(39) CometProject +(41) 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] -(40) ReusedExchange [Reuses operator id: 22] +(42) ReusedExchange [Reuses operator id: 22] Output [1]: [sm_ship_mode_sk#188] -(41) CometBroadcastHashJoin +(43) 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 -(42) CometProject +(44) 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] +(45) 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] -(44) HashAggregate [codegen id : 3] +(46) 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] -(45) Exchange +(47) RowToColumnar +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] + +(48) CometColumnarExchange 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=2] +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, CometColumnarShuffle, [plan_id=2] -(46) HashAggregate [codegen id : 4] +(49) ColumnarToRow [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] + +(50) 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] -(47) Union +(51) Union -(48) HashAggregate [codegen id : 5] +(52) 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] -(49) Exchange +(53) RowToColumnar +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] + +(54) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=3] + +(55) ColumnarToRow [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] -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] -(50) HashAggregate [codegen id : 6] +(56) 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] -(51) TakeOrderedAndProject +(57) 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 (55) -+- * ColumnarToRow (54) - +- CometFilter (53) - +- CometScan parquet spark_catalog.default.date_dim (52) +BroadcastExchange (61) ++- * ColumnarToRow (60) + +- CometFilter (59) + +- CometScan parquet spark_catalog.default.date_dim (58) -(52) Scan parquet spark_catalog.default.date_dim +(58) 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 -(53) CometFilter +(59) 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)) -(54) ColumnarToRow [codegen id : 1] +(60) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#16, d_year#17, d_moy#18] -(55) BroadcastExchange +(61) 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=4] -Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#175 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 31 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 880c38bbf..4c60afa90 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,71 +1,77 @@ 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 (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 (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 (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 (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] - 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,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,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,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,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_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,d_date_sk,d_year,d_moy] - 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_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometFilter [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 - CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - 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 [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [t_time_sk] #6 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_time] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] - CometBroadcastExchange [sm_ship_mode_sk] #7 - CometProject [sm_ship_mode_sk] - CometFilter [sm_ship_mode_sk,sm_carrier] - 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] #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] - 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,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,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,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,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_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,d_date_sk,d_year,d_moy] - 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_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometFilter [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] - 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 - 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 + ColumnarToRow + InputAdapter + CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year] #1 + RowToColumnar + 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 (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] + ColumnarToRow + InputAdapter + CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #2 + RowToColumnar + 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] + 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,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,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,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,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_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,d_date_sk,d_year,d_moy] + 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_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometFilter [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 + CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + 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 [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [t_time_sk] #6 + CometProject [t_time_sk] + CometFilter [t_time_sk,t_time] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] + CometBroadcastExchange [sm_ship_mode_sk] #7 + CometProject [sm_ship_mode_sk] + CometFilter [sm_ship_mode_sk,sm_carrier] + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #8 + RowToColumnar + 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] + 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,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,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,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,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_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,d_date_sk,d_year,d_moy] + 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_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometFilter [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] + 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 + 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 a1624de1c..6213ac10e 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,35 +1,39 @@ == Physical Plan == -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) +TakeOrderedAndProject (35) ++- * Filter (34) + +- Window (33) + +- WindowGroupLimit (32) + +- * ColumnarToRow (31) + +- CometSort (30) + +- CometColumnarExchange (29) + +- RowToColumnar (28) + +- WindowGroupLimit (27) + +- * Sort (26) + +- * HashAggregate (25) + +- * ColumnarToRow (24) + +- CometColumnarExchange (23) + +- RowToColumnar (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 @@ -134,78 +138,90 @@ Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as d 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] -(22) Exchange +(22) RowToColumnar 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] -(23) HashAggregate [codegen id : 2] +(23) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(24) ColumnarToRow [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] + +(25) 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))#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] -(24) Sort [codegen id : 2] +(26) 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 -(25) WindowGroupLimit +(27) 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 -(26) Exchange +(28) RowToColumnar 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] -(27) Sort [codegen id : 3] +(29) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] + +(30) CometSort +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, 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], [i_category#19 ASC NULLS FIRST, sumsales#33 DESC NULLS LAST] + +(31) ColumnarToRow [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 -(28) WindowGroupLimit +(32) 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 -(29) Window +(33) 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] -(30) Filter [codegen id : 4] +(34) 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) -(31) TakeOrderedAndProject +(35) 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 (36) -+- * ColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan parquet spark_catalog.default.date_dim (32) +BroadcastExchange (40) ++- * ColumnarToRow (39) + +- CometProject (38) + +- CometFilter (37) + +- CometScan parquet spark_catalog.default.date_dim (36) -(32) Scan parquet spark_catalog.default.date_dim +(36) 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 -(33) CometFilter +(37) 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)) -(34) CometProject +(38) 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] -(35) ColumnarToRow [codegen id : 1] +(39) ColumnarToRow [codegen id : 1] Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(36) BroadcastExchange +(40) 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 46d9cf4ef..968c7d056 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 @@ -5,43 +5,47 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ Window [sumsales,i_category] WindowGroupLimit [i_category,sumsales] WholeStageCodegen (3) - Sort [i_category,sumsales] + ColumnarToRow InputAdapter - Exchange [i_category] #1 - WindowGroupLimit [i_category,sumsales] - 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 (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] - 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,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] - CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] - CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometFilter [s_store_sk,s_store_id] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometColumnarExchange [i_category] #1 + RowToColumnar + WindowGroupLimit [i_category,sumsales] + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 + RowToColumnar + 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] + 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,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] + CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + 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 850cd5ccb..054c5eb63 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,46 +1,48 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * Project (41) - +- * BroadcastHashJoin Inner BuildRight (40) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * HashAggregate (29) - : : +- Exchange (28) - : : +- * HashAggregate (27) - : : +- * 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) - +- BroadcastExchange (39) - +- * ColumnarToRow (38) - +- CometFilter (37) - +- CometScan parquet spark_catalog.default.customer_address (36) +TakeOrderedAndProject (44) ++- * Project (43) + +- * BroadcastHashJoin Inner BuildRight (42) + :- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * HashAggregate (31) + : : +- * ColumnarToRow (30) + : : +- CometColumnarExchange (29) + : : +- RowToColumnar (28) + : : +- * HashAggregate (27) + : : +- * 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 (35) + : +- * ColumnarToRow (34) + : +- CometFilter (33) + : +- CometScan parquet spark_catalog.default.customer (32) + +- BroadcastExchange (41) + +- * ColumnarToRow (40) + +- CometFilter (39) + +- CometScan parquet spark_catalog.default.customer_address (38) (1) Scan parquet spark_catalog.default.store_sales @@ -173,106 +175,112 @@ Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#6)), partial_sum(Un 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 +(28) RowToColumnar 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 : 4] +(29) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(30) ColumnarToRow [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] + +(31) 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))#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 +(32) Scan parquet spark_catalog.default.customer 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 +(33) CometFilter 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 : 2] +(34) 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 +(35) BroadcastExchange 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 : 4] +(36) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_customer_sk#1] Right keys [1]: [c_customer_sk#34] Join type: Inner Join condition: None -(35) Project [codegen id : 4] +(37) 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 +(38) 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 -(37) CometFilter +(39) 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] +(40) ColumnarToRow [codegen id : 3] Input [2]: [ca_address_sk#38, ca_city#39] -(39) BroadcastExchange +(41) BroadcastExchange Input [2]: [ca_address_sk#38, ca_city#39] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(40) BroadcastHashJoin [codegen id : 4] +(42) 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#39 = bought_city#30) -(41) Project [codegen id : 4] +(43) 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] -(42) TakeOrderedAndProject +(44) 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 (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 +(45) 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 -(44) CometFilter +(46) 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)) -(45) CometProject +(47) CometProject Input [3]: [d_date_sk#11, d_year#12, d_dom#13] Arguments: [d_date_sk#11], [d_date_sk#11] -(46) ColumnarToRow [codegen id : 1] +(48) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(47) BroadcastExchange +(49) BroadcastExchange Input [1]: [d_date_sk#11] 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 78e1b1cce..df6e8541d 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 @@ -5,45 +5,47 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ Project [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name] BroadcastHashJoin [ss_customer_sk,c_customer_sk] 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 (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] - 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_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_address_sk,ca_city] - CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,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_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,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_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,d_date_sk] - CometFilter [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] - 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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_city] - CometScan parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange [hd_demo_sk] #5 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [ca_address_sk,ca_city] #6 - CometFilter [ca_address_sk,ca_city] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] + ColumnarToRow + InputAdapter + CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 + RowToColumnar + 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] + 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_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_address_sk,ca_city] + CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,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_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,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_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,d_date_sk] + CometFilter [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] + 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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange [s_store_sk] #4 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_city] + CometScan parquet spark_catalog.default.store [s_store_sk,s_city] + CometBroadcastExchange [hd_demo_sk] #5 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange [ca_address_sk,ca_city] #6 + CometFilter [ca_address_sk,ca_city] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) 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 b931de292..4991d569f 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,48 +1,50 @@ == Physical Plan == -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) +TakeOrderedAndProject (46) ++- * HashAggregate (45) + +- * ColumnarToRow (44) + +- CometColumnarExchange (43) + +- RowToColumnar (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 @@ -238,50 +240,56 @@ Functions [1]: [partial_count(1)] 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] -(42) Exchange +(42) RowToColumnar 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] -(43) HashAggregate [codegen id : 6] +(43) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=5] + +(44) ColumnarToRow [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] + +(45) 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)#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] -(44) TakeOrderedAndProject +(46) 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 = 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) +BroadcastExchange (51) ++- * ColumnarToRow (50) + +- CometProject (49) + +- CometFilter (48) + +- CometScan parquet spark_catalog.default.date_dim (47) -(45) Scan parquet spark_catalog.default.date_dim +(47) 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 -(46) CometFilter +(48) 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)) -(47) CometProject +(49) CometProject Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(48) ColumnarToRow [codegen id : 1] +(50) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(49) BroadcastExchange +(51) BroadcastExchange Input [1]: [d_date_sk#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-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 c9cc4959b..9e1e3edb6 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,70 +1,72 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cnt1,cnt2,cnt3] 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 (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] - 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,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow + ColumnarToRow + InputAdapter + CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] #1 + RowToColumnar + 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] + 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,cs_ship_customer_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [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] + CometBroadcastExchange [ss_customer_sk] #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,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 InputAdapter - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] - CometFilter [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] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,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 + BroadcastExchange #6 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [cs_ship_customer_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,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 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) + BroadcastExchange #7 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,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 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) + BroadcastExchange #8 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] 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 5755ed1dc..08ea991ad 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 @@ -1,34 +1,36 @@ == Physical Plan == -TakeOrderedAndProject (30) -+- * HashAggregate (29) - +- Exchange (28) - +- * HashAggregate (27) - +- * 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) +TakeOrderedAndProject (32) ++- * HashAggregate (31) + +- * ColumnarToRow (30) + +- CometColumnarExchange (29) + +- RowToColumnar (28) + +- * HashAggregate (27) + +- * 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) (1) Scan parquet spark_catalog.default.store_sales @@ -161,50 +163,56 @@ Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_pr 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 +(28) RowToColumnar 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 : 2] +(29) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(30) ColumnarToRow [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] + +(31) 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)#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 +(32) TakeOrderedAndProject 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 ===== 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 (37) ++- * ColumnarToRow (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) -(31) Scan parquet spark_catalog.default.date_dim +(33) 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 -(32) CometFilter +(34) 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)) -(33) CometProject +(35) CometProject Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(34) ColumnarToRow [codegen id : 1] +(36) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(35) BroadcastExchange +(37) BroadcastExchange Input [1]: [d_date_sk#14] 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 a27daca36..04db30380 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,42 +1,44 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] 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 (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] - ColumnarToRow - InputAdapter - CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - CometBroadcastHashJoin [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,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,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] - CometFilter [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #5 - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [p_promo_sk] #6 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_email,p_channel_event] - CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #1 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] + CometBroadcastHashJoin [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,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,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] + CometProject [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [cd_demo_sk] #3 + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_item_id] #5 + CometFilter [i_item_sk,i_item_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange [p_promo_sk] #6 + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_email,p_channel_event] + 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 0a6934960..b55d8224b 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,49 +1,55 @@ == Physical Plan == -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) +TakeOrderedAndProject (51) ++- * Project (50) + +- Window (49) + +- * ColumnarToRow (48) + +- CometSort (47) + +- CometColumnarExchange (46) + +- RowToColumnar (45) + +- * HashAggregate (44) + +- * ColumnarToRow (43) + +- CometColumnarExchange (42) + +- RowToColumnar (41) + +- * HashAggregate (40) + +- * Expand (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * 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 (36) + +- * BroadcastHashJoin LeftSemi BuildRight (35) + :- * ColumnarToRow (12) + : +- CometFilter (11) + : +- CometScan parquet spark_catalog.default.store (10) + +- BroadcastExchange (34) + +- * Project (33) + +- * Filter (32) + +- Window (31) + +- WindowGroupLimit (30) + +- * Sort (29) + +- * HashAggregate (28) + +- * ColumnarToRow (27) + +- CometColumnarExchange (26) + +- RowToColumnar (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 @@ -161,132 +167,150 @@ Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] Aggregate Attributes [1]: [sum#17] Results [2]: [s_state#15, sum#18] -(25) Exchange +(25) RowToColumnar Input [2]: [s_state#15, sum#18] -Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(26) HashAggregate [codegen id : 2] +(26) CometColumnarExchange +Input [2]: [s_state#15, sum#18] +Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(27) ColumnarToRow [codegen id : 2] +Input [2]: [s_state#15, sum#18] + +(28) 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] +(29) 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 -(28) WindowGroupLimit +(30) 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 +(31) 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] +(32) 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] +(33) Project [codegen id : 3] Output [1]: [s_state#15] Input [4]: [s_state#15, _w0#20, s_state#15, ranking#21] -(32) BroadcastExchange +(34) BroadcastExchange Input [1]: [s_state#15] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] -(33) BroadcastHashJoin [codegen id : 4] +(35) BroadcastHashJoin [codegen id : 4] Left keys [1]: [s_state#9] Right keys [1]: [s_state#15] Join type: LeftSemi Join condition: None -(34) BroadcastExchange +(36) 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] -(35) BroadcastHashJoin [codegen id : 5] +(37) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_store_sk#1] Right keys [1]: [s_store_sk#7] Join type: Inner Join condition: None -(36) Project [codegen id : 5] +(38) 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] -(37) Expand [codegen id : 5] +(39) 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] -(38) HashAggregate [codegen id : 5] +(40) 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#25] Results [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] -(39) Exchange +(41) RowToColumnar +Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] + +(42) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=4] + +(43) ColumnarToRow [codegen id : 6] 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] -(40) HashAggregate [codegen id : 6] +(44) 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))#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] -(41) Exchange +(45) RowToColumnar +Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] + +(46) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=5] + +(47) CometSort 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] +Arguments: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32], [_w1#31 ASC NULLS FIRST, _w2#32 ASC NULLS FIRST, _w0#30 DESC NULLS LAST] -(42) Sort [codegen id : 7] +(48) ColumnarToRow [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 -(43) Window +(49) 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] -(44) Project [codegen id : 8] +(50) 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] -(45) TakeOrderedAndProject +(51) 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 (50) -+- * ColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan parquet spark_catalog.default.date_dim (46) +BroadcastExchange (56) ++- * ColumnarToRow (55) + +- CometProject (54) + +- CometFilter (53) + +- CometScan parquet spark_catalog.default.date_dim (52) -(46) Scan parquet spark_catalog.default.date_dim +(52) 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 -(47) CometFilter +(53) 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)) -(48) CometProject +(54) CometProject Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(49) ColumnarToRow [codegen id : 1] +(55) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(50) BroadcastExchange +(56) BroadcastExchange Input [1]: [d_date_sk#5] 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/q70/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt index 4a6dffc08..b8e21d9d4 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 @@ -4,69 +4,75 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count InputAdapter Window [_w0,_w1,_w2] WholeStageCodegen (7) - Sort [_w1,_w2,_w0] + ColumnarToRow InputAdapter - Exchange [_w1,_w2] #1 - 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 (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] - ColumnarToRow - InputAdapter - CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - BroadcastHashJoin [s_state,s_state] + CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] + CometColumnarExchange [_w1,_w2] #1 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 + RowToColumnar + 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] ColumnarToRow InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + CometProject [ss_store_sk,ss_net_profit] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - Project [s_state] - Filter [ranking] + BroadcastExchange #5 + WholeStageCodegen (4) + BroadcastHashJoin [s_state,s_state] + ColumnarToRow InputAdapter - Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (2) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - InputAdapter - Exchange [s_state] #7 - WholeStageCodegen (1) - HashAggregate [s_state,ss_net_profit] [sum,sum] + CometFilter [s_store_sk,s_county,s_state] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + Project [s_state] + Filter [ranking] + InputAdapter + Window [_w0,s_state] + WindowGroupLimit [s_state,_w0] + WholeStageCodegen (2) + Sort [s_state,_w0] + HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] ColumnarToRow InputAdapter - CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] - CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #8 - CometFilter [s_store_sk,s_state] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [d_date_sk] #4 + CometColumnarExchange [s_state] #7 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [s_state,ss_net_profit] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [ss_net_profit,s_state] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] + CometProject [ss_net_profit,ss_sold_date_sk,s_state] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [s_store_sk,s_state] #8 + CometFilter [s_store_sk,s_state] + 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 8ec2c5c5a..137da5e62 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,41 +1,45 @@ == Physical Plan == -* 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) +* ColumnarToRow (41) ++- CometSort (40) + +- CometColumnarExchange (39) + +- RowToColumnar (38) + +- * HashAggregate (37) + +- * ColumnarToRow (36) + +- CometColumnarExchange (35) + +- RowToColumnar (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 @@ -197,54 +201,66 @@ 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 +(34) RowToColumnar 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] +(35) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(36) ColumnarToRow [codegen id : 2] +Input [5]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, sum#39] + +(37) 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 +(38) RowToColumnar +Input [5]: [brand_id#41, brand#42, t_hour#35, t_minute#36, ext_price#43] + +(39) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] + +(40) CometSort 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] +Arguments: [brand_id#41, brand#42, t_hour#35, t_minute#36, ext_price#43], [ext_price#43 DESC NULLS LAST, brand_id#41 ASC NULLS FIRST] -(37) Sort [codegen id : 3] +(41) ColumnarToRow [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 = 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) +BroadcastExchange (46) ++- * ColumnarToRow (45) + +- CometProject (44) + +- CometFilter (43) + +- CometScan parquet spark_catalog.default.date_dim (42) -(38) Scan parquet spark_catalog.default.date_dim +(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_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct -(39) CometFilter +(43) 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)) -(40) CometProject +(44) CometProject Input [3]: [d_date_sk#10, d_year#11, d_moy#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(41) ColumnarToRow [codegen id : 1] +(45) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(42) BroadcastExchange +(46) BroadcastExchange Input [1]: [d_date_sk#10] 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/q71/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/simplified.txt index a4be53684..a4649a1c3 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,53 +1,57 @@ WholeStageCodegen (3) - Sort [ext_price,brand_id] + ColumnarToRow InputAdapter - Exchange [ext_price,brand_id] #1 - 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 (1) - HashAggregate [i_brand,i_brand_id,t_hour,t_minute,ext_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [i_brand_id,i_brand,ext_price,t_hour,t_minute] - CometBroadcastHashJoin [i_brand_id,i_brand,ext_price,time_sk,t_time_sk,t_hour,t_minute] - CometProject [i_brand_id,i_brand,ext_price,time_sk] - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_brand,ext_price,sold_item_sk,time_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometUnion [ext_price,sold_item_sk,time_sk] - CometProject [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] - CometFilter [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - 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_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] - CometFilter [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometFilter [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_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 [t_time_sk,t_hour,t_minute] #6 - CometProject [t_time_sk,t_hour,t_minute] - CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] + CometSort [brand_id,brand,t_hour,t_minute,ext_price] + CometColumnarExchange [ext_price,brand_id] #1 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum] [sum(UnscaledValue(ext_price)),brand_id,brand,ext_price,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_brand,i_brand_id,t_hour,t_minute] #2 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [i_brand,i_brand_id,t_hour,t_minute,ext_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [i_brand_id,i_brand,ext_price,t_hour,t_minute] + CometBroadcastHashJoin [i_brand_id,i_brand,ext_price,time_sk,t_time_sk,t_hour,t_minute] + CometProject [i_brand_id,i_brand,ext_price,time_sk] + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_brand,ext_price,sold_item_sk,time_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 + CometProject [i_item_sk,i_brand_id,i_brand] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] + CometUnion [ext_price,sold_item_sk,time_sk] + CometProject [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] [ext_price,sold_item_sk,time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_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_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + 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_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_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 [t_time_sk,t_hour,t_minute] #6 + CometProject [t_time_sk,t_hour,t_minute] + CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] + 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 26edd145c..2f6da7d32 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,71 +1,72 @@ == Physical Plan == -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) +* ColumnarToRow (68) ++- CometTakeOrderedAndProject (67) + +- CometHashAggregate (66) + +- CometColumnarExchange (65) + +- CometHashAggregate (64) + +- CometProject (63) + +- CometSortMergeJoin (62) + :- CometSort (56) + : +- CometColumnarExchange (55) + : +- RowToColumnar (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) + +- CometSort (61) + +- CometColumnarExchange (60) + +- CometProject (59) + +- CometFilter (58) + +- CometScan parquet spark_catalog.default.catalog_returns (57) (1) Scan parquet spark_catalog.default.catalog_sales @@ -320,101 +321,99 @@ Join condition: None 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#30] -(54) Exchange +(54) RowToColumnar 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=3] -(55) Sort [codegen id : 4] +(55) CometColumnarExchange 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 +Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(56) Scan parquet spark_catalog.default.catalog_returns +(56) CometSort +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, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST] + +(57) 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 -(57) CometFilter +(58) 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)) -(58) CometProject +(59) 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] -(59) ColumnarToRow [codegen id : 5] +(60) CometColumnarExchange Input [2]: [cr_item_sk#31, cr_order_number#32] +Arguments: hashpartitioning(cr_item_sk#31, cr_order_number#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(60) Exchange +(61) CometSort 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] +Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31 ASC NULLS FIRST, cr_order_number#32 ASC NULLS FIRST] -(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 +(62) CometSortMergeJoin +Left output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +Right output [2]: [cr_item_sk#31, cr_order_number#32] +Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#31, cr_order_number#32], LeftOuter -(62) SortMergeJoin [codegen id : 7] -Left keys [2]: [cs_item_sk#4, cs_order_number#6] -Right keys [2]: [cr_item_sk#31, cr_order_number#32] -Join type: LeftOuter -Join condition: None - -(63) Project [codegen id : 7] -Output [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +(63) CometProject 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] +Arguments: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -(64) HashAggregate [codegen id : 7] +(64) CometHashAggregate 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#34] -Results [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#35] -(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] +(65) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=5] -(66) HashAggregate [codegen id : 8] -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#35] +(66) CometHashAggregate +Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] Functions [1]: [count(1)] -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] -(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] +(67) CometTakeOrderedAndProject +Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#37 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#35,promo#36,total_cnt#37]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37], 100, [total_cnt#37 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#35, promo#36, total_cnt#37] + +(68) ColumnarToRow [codegen id : 4] +Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -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 +(69) 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 -(69) CometFilter +(70) 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)) -(70) CometProject +(71) 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] -(71) ColumnarToRow [codegen id : 1] +(72) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -(72) BroadcastExchange +(73) 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=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 6cba2d0e6..1022e1a90 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,94 +1,86 @@ -TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo] - 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 (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 (4) - Sort [cs_item_sk,cs_order_number] - InputAdapter - Exchange [cs_item_sk,cs_order_number] #2 - 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] - 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 [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,d_date_sk,d_week_seq] - 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_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,d_date_sk,d_date,d_week_seq] - 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_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,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_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,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_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_sk,i_item_desc] - 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 [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,w_warehouse_sk,w_warehouse_name] - 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_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,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometFilter [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] - 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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [i_item_sk,i_item_desc] #6 - CometFilter [i_item_sk,i_item_desc] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange [cd_demo_sk] #7 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange [hd_demo_sk] #8 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange [d_date_sk,d_week_seq] #10 - CometFilter [d_date_sk,d_week_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] +WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] + CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count,count(1)] + CometColumnarExchange [i_item_desc,w_warehouse_name,d_week_seq] #1 + CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] + CometProject [w_warehouse_name,i_item_desc,d_week_seq] + CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] + CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + CometColumnarExchange [cs_item_sk,cs_order_number] #2 + RowToColumnar + 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] + 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 [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,d_date_sk,d_week_seq] + 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_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,d_date_sk,d_date,d_week_seq] + 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_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,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_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,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_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_sk,i_item_desc] + 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 [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,w_warehouse_sk,w_warehouse_name] + 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_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,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometFilter [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] + 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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange [i_item_sk,i_item_desc] #6 + CometFilter [i_item_sk,i_item_desc] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + CometBroadcastExchange [cd_demo_sk] #7 + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + CometBroadcastExchange [hd_demo_sk] #8 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] + CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + CometBroadcastExchange [d_date_sk,d_week_seq] #10 + CometFilter [d_date_sk,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter - BroadcastExchange #12 - WholeStageCodegen (2) + BroadcastExchange #11 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk] - InputAdapter - WholeStageCodegen (6) - Sort [cr_item_sk,cr_order_number] - InputAdapter - Exchange [cr_item_sk,cr_order_number] #13 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk] + CometSort [cr_item_sk,cr_order_number] + CometColumnarExchange [cr_item_sk,cr_order_number] #13 + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] 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 88dcba0fb..6ea426978 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,37 +1,36 @@ == Physical Plan == -* 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) +* ColumnarToRow (32) ++- CometSort (31) + +- CometColumnarExchange (30) + +- CometProject (29) + +- CometBroadcastHashJoin (28) + :- CometFilter (24) + : +- CometHashAggregate (23) + : +- CometColumnarExchange (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) + +- CometBroadcastExchange (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.customer (25) (1) Scan parquet spark_catalog.default.store_sales @@ -135,90 +134,84 @@ 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)] -(22) ColumnarToRow [codegen id : 1] +(22) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(24) HashAggregate [codegen id : 3] +(23) CometHashAggregate 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)#17] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#17 AS cnt#18] -(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)) +(24) CometFilter +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] +Condition : ((cnt#17 >= 1) AND (cnt#17 <= 5)) -(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] +(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] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(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) +(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) -(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] +(27) CometBroadcastExchange +Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Arguments: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -(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] +(28) CometBroadcastHashJoin +Left output [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] +Right output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight -(30) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#19] -Join type: Inner -Join condition: None +(29) CometProject +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] +Arguments: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17], [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -(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] +(30) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] -(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] +(31) CometSort +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, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17], [cnt#17 DESC NULLS LAST] -(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 +(32) ColumnarToRow [codegen id : 1] +Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (38) -+- * ColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.date_dim (34) +BroadcastExchange (37) ++- * ColumnarToRow (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) -(34) Scan parquet spark_catalog.default.date_dim +(33) 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 -(35) CometFilter +(34) 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)) -(36) CometProject +(35) CometProject Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(37) ColumnarToRow [codegen id : 1] +(36) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(38) BroadcastExchange +(37) BroadcastExchange Input [1]: [d_date_sk#7] -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/q73/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt index c91c4cf47..5fb8a197a 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,50 +1,42 @@ -WholeStageCodegen (4) - Sort [cnt] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [cnt] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,count] - CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] - CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] - CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #5 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange [hd_demo_sk] #6 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + CometColumnarExchange [cnt] #1 + CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometFilter [ss_ticket_number,ss_customer_sk,cnt] + CometHashAggregate [ss_ticket_number,ss_customer_sk,cnt,count,count(1)] + CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] + CometProject [ss_customer_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange [s_store_sk] #5 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_county] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county] + CometBroadcastExchange [hd_demo_sk] #6 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] #7 + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] 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 d5eb481f1..2fad5f7a5 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 @@ -1,75 +1,83 @@ == Physical Plan == -TakeOrderedAndProject (71) -+- * Project (70) - +- * BroadcastHashJoin Inner BuildRight (69) - :- * 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) - +- * 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) +TakeOrderedAndProject (79) ++- * Project (78) + +- * BroadcastHashJoin Inner BuildRight (77) + :- * Project (59) + : +- * BroadcastHashJoin Inner BuildRight (58) + : :- * BroadcastHashJoin Inner BuildRight (39) + : : :- * Filter (19) + : : : +- * HashAggregate (18) + : : : +- * ColumnarToRow (17) + : : : +- CometColumnarExchange (16) + : : : +- RowToColumnar (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 (38) + : : +- * HashAggregate (37) + : : +- * ColumnarToRow (36) + : : +- CometColumnarExchange (35) + : : +- RowToColumnar (34) + : : +- * HashAggregate (33) + : : +- * ColumnarToRow (32) + : : +- CometProject (31) + : : +- CometBroadcastHashJoin (30) + : : :- CometProject (26) + : : : +- CometBroadcastHashJoin (25) + : : : :- CometFilter (21) + : : : : +- CometScan parquet spark_catalog.default.customer (20) + : : : +- CometBroadcastExchange (24) + : : : +- CometFilter (23) + : : : +- CometScan parquet spark_catalog.default.store_sales (22) + : : +- CometBroadcastExchange (29) + : : +- CometFilter (28) + : : +- CometScan parquet spark_catalog.default.date_dim (27) + : +- BroadcastExchange (57) + : +- * Filter (56) + : +- * HashAggregate (55) + : +- * ColumnarToRow (54) + : +- CometColumnarExchange (53) + : +- RowToColumnar (52) + : +- * HashAggregate (51) + : +- * ColumnarToRow (50) + : +- CometProject (49) + : +- CometBroadcastHashJoin (48) + : :- CometProject (46) + : : +- CometBroadcastHashJoin (45) + : : :- CometFilter (41) + : : : +- CometScan parquet spark_catalog.default.customer (40) + : : +- CometBroadcastExchange (44) + : : +- CometFilter (43) + : : +- CometScan parquet spark_catalog.default.web_sales (42) + : +- ReusedExchange (47) + +- BroadcastExchange (76) + +- * HashAggregate (75) + +- * ColumnarToRow (74) + +- CometColumnarExchange (73) + +- RowToColumnar (72) + +- * HashAggregate (71) + +- * ColumnarToRow (70) + +- CometProject (69) + +- CometBroadcastHashJoin (68) + :- CometProject (66) + : +- CometBroadcastHashJoin (65) + : :- CometFilter (61) + : : +- CometScan parquet spark_catalog.default.customer (60) + : +- CometBroadcastExchange (64) + : +- CometFilter (63) + : +- CometScan parquet spark_catalog.default.web_sales (62) + +- ReusedExchange (67) (1) Scan parquet spark_catalog.default.customer @@ -142,33 +150,39 @@ 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] -(15) Exchange +(15) RowToColumnar 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=1] -(16) HashAggregate [codegen id : 8] +(16) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(17) ColumnarToRow [codegen id : 8] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] + +(18) 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] -(17) Filter [codegen id : 8] +(19) Filter [codegen id : 8] Input [2]: [customer_id#14, year_total#15] Condition : (isnotnull(year_total#15) AND (year_total#15 > 0.00)) -(18) Scan parquet spark_catalog.default.customer +(20) 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 -(19) CometFilter +(21) 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)) -(20) Scan parquet spark_catalog.default.store_sales +(22) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] Batched: true Location: InMemoryFileIndex [] @@ -176,90 +190,96 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(21) CometFilter +(23) CometFilter Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] Condition : isnotnull(ss_customer_sk#20) -(22) CometBroadcastExchange +(24) 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) CometBroadcastHashJoin +(25) 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) CometProject +(26) 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] -(25) Scan parquet spark_catalog.default.date_dim +(27) 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 -(26) CometFilter +(28) 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 +(29) CometBroadcastExchange Input [2]: [d_date_sk#24, d_year#25] Arguments: [d_date_sk#24, d_year#25] -(28) CometBroadcastHashJoin +(30) 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 +(31) 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] +(32) 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] -(31) HashAggregate [codegen id : 2] +(33) 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] -(32) Exchange +(34) RowToColumnar +Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] + +(35) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] + +(36) ColumnarToRow [codegen id : 3] 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=2] -(33) HashAggregate [codegen id : 3] +(37) 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] -(34) BroadcastExchange +(38) 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=3] -(35) BroadcastHashJoin [codegen id : 8] +(39) BroadcastHashJoin [codegen id : 8] Left keys [1]: [customer_id#14] Right keys [1]: [customer_id#28] Join type: Inner Join condition: None -(36) Scan parquet spark_catalog.default.customer +(40) 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 -(37) CometFilter +(41) 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)) -(38) Scan parquet spark_catalog.default.web_sales +(42) 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 [] @@ -267,86 +287,92 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(39) CometFilter +(43) CometFilter Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] Condition : isnotnull(ws_bill_customer_sk#36) -(40) CometBroadcastExchange +(44) CometBroadcastExchange Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] Arguments: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] -(41) CometBroadcastHashJoin +(45) 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) CometProject +(46) 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: 10] +(47) ReusedExchange [Reuses operator id: 10] Output [2]: [d_date_sk#40, d_year#41] -(44) CometBroadcastHashJoin +(48) 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) CometProject +(49) 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) ColumnarToRow [codegen id : 4] +(50) 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] +(51) 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] -(48) Exchange +(52) RowToColumnar +Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] + +(53) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=4] + +(54) ColumnarToRow [codegen id : 5] 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=4] -(49) HashAggregate [codegen id : 5] +(55) 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] -(50) Filter [codegen id : 5] +(56) Filter [codegen id : 5] Input [2]: [customer_id#45, year_total#46] Condition : (isnotnull(year_total#46) AND (year_total#46 > 0.00)) -(51) BroadcastExchange +(57) BroadcastExchange Input [2]: [customer_id#45, year_total#46] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(52) BroadcastHashJoin [codegen id : 8] +(58) BroadcastHashJoin [codegen id : 8] Left keys [1]: [customer_id#14] Right keys [1]: [customer_id#45] Join type: Inner Join condition: None -(53) Project [codegen id : 8] +(59) 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] -(54) Scan parquet spark_catalog.default.customer +(60) 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 -(55) CometFilter +(61) 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)) -(56) Scan parquet spark_catalog.default.web_sales +(62) 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 Location: InMemoryFileIndex [] @@ -354,128 +380,134 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(57) CometFilter +(63) CometFilter Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] Condition : isnotnull(ws_bill_customer_sk#51) -(58) CometBroadcastExchange +(64) 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) CometBroadcastHashJoin +(65) 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) CometProject +(66) 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] -(61) ReusedExchange [Reuses operator id: 27] +(67) ReusedExchange [Reuses operator id: 29] Output [2]: [d_date_sk#55, d_year#56] -(62) CometBroadcastHashJoin +(68) 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 -(63) CometProject +(69) 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] -(64) ColumnarToRow [codegen id : 6] +(70) 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] +(71) 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))] Aggregate Attributes [1]: [sum#57] Results [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] -(66) Exchange +(72) RowToColumnar +Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] + +(73) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=6] + +(74) ColumnarToRow [codegen id : 7] 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=6] -(67) HashAggregate [codegen id : 7] +(75) 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))] 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] -(68) BroadcastExchange +(76) BroadcastExchange Input [2]: [customer_id#59, year_total#60] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] -(69) BroadcastHashJoin [codegen id : 8] +(77) 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 : 8] +(78) 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] -(71) TakeOrderedAndProject +(79) 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] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (75) -+- * ColumnarToRow (74) - +- CometFilter (73) - +- CometScan parquet spark_catalog.default.date_dim (72) +BroadcastExchange (83) ++- * ColumnarToRow (82) + +- CometFilter (81) + +- CometScan parquet spark_catalog.default.date_dim (80) -(72) Scan parquet spark_catalog.default.date_dim +(80) 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 -(73) CometFilter +(81) 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)) -(74) ColumnarToRow [codegen id : 1] +(82) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_year#10] -(75) BroadcastExchange +(83) BroadcastExchange Input [2]: [d_date_sk#9, d_year#10] 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) -+- * ColumnarToRow (78) - +- CometFilter (77) - +- CometScan parquet spark_catalog.default.date_dim (76) +Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 +BroadcastExchange (87) ++- * ColumnarToRow (86) + +- CometFilter (85) + +- CometScan parquet spark_catalog.default.date_dim (84) -(76) Scan parquet spark_catalog.default.date_dim +(84) 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 -(77) CometFilter +(85) 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)) -(78) ColumnarToRow [codegen id : 1] +(86) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#24, d_year#25] -(79) BroadcastExchange +(87) BroadcastExchange Input [2]: [d_date_sk#24, d_year#25] 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#38 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 42 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 +Subquery:4 Hosting operator id = 62 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 a7aa864ce..0cc944868 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 @@ -7,38 +7,11 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] BroadcastHashJoin [customer_id,customer_id] Filter [year_total] 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 (1) - 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 [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_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_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - 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 + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 + RowToColumnar + WholeStageCodegen (1) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] ColumnarToRow InputAdapter @@ -48,59 +21,94 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 + CometBroadcastExchange [d_date_sk,d_year] #4 CometFilter [d_date_sk,d_year] 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #6 + RowToColumnar + 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 [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter 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] #11 - WholeStageCodegen (4) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 - CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_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 + ColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #11 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_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 #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] #14 - WholeStageCodegen (6) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 - CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_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 + ColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #14 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_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 1c61a76d1..08fb52740 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,136 +1,137 @@ == Physical Plan == -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) +TakeOrderedAndProject (133) ++- * Project (132) + +- * SortMergeJoin Inner (131) + :- * ColumnarToRow (71) + : +- CometSort (70) + : +- CometColumnarExchange (69) + : +- RowToColumnar (68) + : +- * Filter (67) + : +- * HashAggregate (66) + : +- * ColumnarToRow (65) + : +- CometColumnarExchange (64) + : +- RowToColumnar (63) + : +- * HashAggregate (62) + : +- * ColumnarToRow (61) + : +- CometHashAggregate (60) + : +- CometColumnarExchange (59) + : +- CometHashAggregate (58) + : +- CometUnion (57) + : :- CometProject (22) + : : +- CometSortMergeJoin (21) + : : :- CometSort (15) + : : : +- CometColumnarExchange (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) + : : +- CometSort (20) + : : +- CometColumnarExchange (19) + : : +- CometProject (18) + : : +- CometFilter (17) + : : +- CometScan parquet spark_catalog.default.catalog_returns (16) + : :- CometProject (39) + : : +- CometSortMergeJoin (38) + : : :- CometSort (32) + : : : +- CometColumnarExchange (31) + : : : +- CometProject (30) + : : : +- CometBroadcastHashJoin (29) + : : : :- CometProject (27) + : : : : +- CometBroadcastHashJoin (26) + : : : : :- CometFilter (24) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (23) + : : : : +- ReusedExchange (25) + : : : +- ReusedExchange (28) + : : +- CometSort (37) + : : +- CometColumnarExchange (36) + : : +- CometProject (35) + : : +- CometFilter (34) + : : +- CometScan parquet spark_catalog.default.store_returns (33) + : +- CometProject (56) + : +- CometSortMergeJoin (55) + : :- CometSort (49) + : : +- CometColumnarExchange (48) + : : +- CometProject (47) + : : +- CometBroadcastHashJoin (46) + : : :- CometProject (44) + : : : +- CometBroadcastHashJoin (43) + : : : :- CometFilter (41) + : : : : +- CometScan parquet spark_catalog.default.web_sales (40) + : : : +- ReusedExchange (42) + : : +- ReusedExchange (45) + : +- CometSort (54) + : +- CometColumnarExchange (53) + : +- CometProject (52) + : +- CometFilter (51) + : +- CometScan parquet spark_catalog.default.web_returns (50) + +- * ColumnarToRow (130) + +- CometSort (129) + +- CometColumnarExchange (128) + +- RowToColumnar (127) + +- * Filter (126) + +- * HashAggregate (125) + +- * ColumnarToRow (124) + +- CometColumnarExchange (123) + +- RowToColumnar (122) + +- * HashAggregate (121) + +- * ColumnarToRow (120) + +- CometHashAggregate (119) + +- CometColumnarExchange (118) + +- CometHashAggregate (117) + +- CometUnion (116) + :- CometProject (87) + : +- CometSortMergeJoin (86) + : :- CometSort (83) + : : +- CometColumnarExchange (82) + : : +- CometProject (81) + : : +- CometBroadcastHashJoin (80) + : : :- CometProject (76) + : : : +- CometBroadcastHashJoin (75) + : : : :- CometFilter (73) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (72) + : : : +- ReusedExchange (74) + : : +- CometBroadcastExchange (79) + : : +- CometFilter (78) + : : +- CometScan parquet spark_catalog.default.date_dim (77) + : +- CometSort (85) + : +- ReusedExchange (84) + :- CometProject (101) + : +- CometSortMergeJoin (100) + : :- CometSort (97) + : : +- CometColumnarExchange (96) + : : +- CometProject (95) + : : +- CometBroadcastHashJoin (94) + : : :- CometProject (92) + : : : +- CometBroadcastHashJoin (91) + : : : :- CometFilter (89) + : : : : +- CometScan parquet spark_catalog.default.store_sales (88) + : : : +- ReusedExchange (90) + : : +- ReusedExchange (93) + : +- CometSort (99) + : +- ReusedExchange (98) + +- CometProject (115) + +- CometSortMergeJoin (114) + :- CometSort (111) + : +- CometColumnarExchange (110) + : +- CometProject (109) + : +- CometBroadcastHashJoin (108) + : :- CometProject (106) + : : +- CometBroadcastHashJoin (105) + : : :- CometFilter (103) + : : : +- CometScan parquet spark_catalog.default.web_sales (102) + : : +- ReusedExchange (104) + : +- ReusedExchange (107) + +- CometSort (113) + +- ReusedExchange (112) (1) Scan parquet spark_catalog.default.catalog_sales @@ -197,54 +198,47 @@ Arguments: [cs_sold_date_sk#5], [d_date_sk#13], Inner, BuildRight 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] +(14) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(15) Exchange +(15) CometSort 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=1] +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_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST] -(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 - -(17) Scan parquet spark_catalog.default.catalog_returns +(16) 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 -(18) CometFilter +(17) 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)) -(19) CometProject +(18) 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] -(20) ColumnarToRow [codegen id : 3] -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] - -(21) Exchange +(19) CometColumnarExchange 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=2] +Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(22) Sort [codegen id : 4] +(20) CometSort 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 +Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST] -(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 +(21) CometSortMergeJoin +Left 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] +Right output [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] +Arguments: [cs_order_number#2, cs_item_sk#1], [cr_order_number#16, cr_item_sk#15], LeftOuter -(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] +(22) CometProject 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] +Arguments: [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], [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] -(25) Scan parquet spark_catalog.default.store_sales +(23) 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 [] @@ -252,82 +246,75 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(26) CometFilter +(24) 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) -(27) ReusedExchange [Reuses operator id: 6] +(25) 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) CometBroadcastHashJoin +(26) 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) CometProject +(27) 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: 11] +(28) ReusedExchange [Reuses operator id: 11] Output [2]: [d_date_sk#33, d_year#34] -(31) CometBroadcastHashJoin +(29) 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) CometProject +(30) 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] - -(34) Exchange +(31) CometColumnarExchange 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=3] +Arguments: hashpartitioning(ss_ticket_number#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(35) Sort [codegen id : 7] +(32) CometSort 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 +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_ticket_number#23 ASC NULLS FIRST, ss_item_sk#22 ASC NULLS FIRST] -(36) Scan parquet spark_catalog.default.store_returns +(33) 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 -(37) CometFilter +(34) 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)) -(38) CometProject +(35) 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] -(39) ColumnarToRow [codegen id : 8] +(36) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=4] -(40) Exchange +(37) CometSort 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=4] +Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST] -(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 - -(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 +(38) CometSortMergeJoin +Left 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] +Right output [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] +Arguments: [ss_ticket_number#23, ss_item_sk#22], [sr_ticket_number#36, sr_item_sk#35], LeftOuter -(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] +(39) CometProject 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] +Arguments: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41], [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] -(44) Scan parquet spark_catalog.default.web_sales +(40) 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 [] @@ -335,132 +322,139 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(45) CometFilter +(41) 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) -(46) ReusedExchange [Reuses operator id: 6] +(42) 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) CometBroadcastHashJoin +(43) 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) CometProject +(44) 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: 11] +(45) ReusedExchange [Reuses operator id: 11] Output [2]: [d_date_sk#53, d_year#54] -(50) CometBroadcastHashJoin +(46) 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) CometProject +(47) 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) ColumnarToRow [codegen id : 11] +(48) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=5] -(53) Exchange +(49) CometSort 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] +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_order_number#43 ASC NULLS FIRST, ws_item_sk#42 ASC NULLS FIRST] -(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 - -(55) Scan parquet spark_catalog.default.web_returns +(50) 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 -(56) CometFilter +(51) 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)) -(57) CometProject +(52) 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] -(58) ColumnarToRow [codegen id : 13] +(53) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=6] -(59) Exchange +(54) CometSort 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=6] +Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST] -(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 +(55) CometSortMergeJoin +Left 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] +Right output [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] +Arguments: [ws_order_number#43, ws_item_sk#42], [wr_order_number#56, wr_item_sk#55], LeftOuter -(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 - -(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] +(56) CometProject 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] +Arguments: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61], [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] -(63) Union +(57) CometUnion +Child 0 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] +Child 1 Input [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41] +Child 2 Input [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61] -(64) HashAggregate [codegen id : 16] +(58) CometHashAggregate 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] -(65) Exchange +(59) CometColumnarExchange 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=7] +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, CometColumnarShuffle, [plan_id=7] -(66) HashAggregate [codegen id : 17] +(60) CometHashAggregate 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] -(67) HashAggregate [codegen id : 17] +(61) ColumnarToRow [codegen id : 1] +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] + +(62) HashAggregate [codegen id : 1] 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] -(68) Exchange +(63) RowToColumnar +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] + +(64) CometColumnarExchange 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=8] +Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(69) HashAggregate [codegen id : 18] +(65) ColumnarToRow [codegen id : 2] +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] + +(66) HashAggregate [codegen id : 2] 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] -(70) Filter [codegen id : 18] +(67) Filter [codegen id : 2] 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) -(71) Exchange +(68) RowToColumnar +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] + +(69) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=9] + +(70) CometSort 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=9] +Arguments: [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], [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] -(72) Sort [codegen id : 19] +(71) ColumnarToRow [codegen id : 3] 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 -(73) Scan parquet spark_catalog.default.catalog_sales +(72) 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 [] @@ -468,75 +462,71 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#74), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(74) CometFilter +(73) 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) -(75) ReusedExchange [Reuses operator id: 6] +(74) 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) CometBroadcastHashJoin +(75) 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) CometProject +(76) 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) Scan parquet spark_catalog.default.date_dim +(77) 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) CometFilter +(78) 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) CometBroadcastExchange +(79) CometBroadcastExchange Input [2]: [d_date_sk#81, d_year#82] Arguments: [d_date_sk#81, d_year#82] -(81) CometBroadcastHashJoin +(80) 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 +(81) 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] - -(84) Exchange +(82) CometColumnarExchange 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=10] +Arguments: hashpartitioning(cs_order_number#71, cs_item_sk#70, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(85) Sort [codegen id : 21] +(83) CometSort 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 +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_order_number#71 ASC NULLS FIRST, cs_item_sk#70 ASC NULLS FIRST] -(86) ReusedExchange [Reuses operator id: 21] +(84) ReusedExchange [Reuses operator id: 19] Output [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] -(87) Sort [codegen id : 23] +(85) CometSort 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 +Arguments: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86], [cr_order_number#84 ASC NULLS FIRST, cr_item_sk#83 ASC NULLS FIRST] -(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) CometSortMergeJoin +Left 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] +Right output [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] +Arguments: [cs_order_number#71, cs_item_sk#70], [cr_order_number#84, cr_item_sk#83], LeftOuter -(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] +(87) CometProject 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] +Arguments: [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], [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] -(90) Scan parquet spark_catalog.default.store_sales +(88) 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 [] @@ -544,63 +534,59 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#93), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(91) CometFilter +(89) 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) -(92) ReusedExchange [Reuses operator id: 6] +(90) 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] -(93) CometBroadcastHashJoin +(91) 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 -(94) CometProject +(92) 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] -(95) ReusedExchange [Reuses operator id: 80] +(93) ReusedExchange [Reuses operator id: 79] Output [2]: [d_date_sk#100, d_year#101] -(96) CometBroadcastHashJoin +(94) 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 -(97) CometProject +(95) 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] -(98) ColumnarToRow [codegen id : 25] +(96) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=11] -(99) Exchange +(97) CometSort 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] +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_ticket_number#90 ASC NULLS FIRST, ss_item_sk#89 ASC NULLS FIRST] -(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 - -(101) ReusedExchange [Reuses operator id: 40] +(98) ReusedExchange [Reuses operator id: 36] Output [4]: [sr_item_sk#102, sr_ticket_number#103, sr_return_quantity#104, sr_return_amt#105] -(102) Sort [codegen id : 28] +(99) CometSort 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 +Arguments: [sr_item_sk#102, sr_ticket_number#103, sr_return_quantity#104, sr_return_amt#105], [sr_ticket_number#103 ASC NULLS FIRST, sr_item_sk#102 ASC NULLS FIRST] -(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 +(100) CometSortMergeJoin +Left 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] +Right output [4]: [sr_item_sk#102, sr_ticket_number#103, sr_return_quantity#104, sr_return_amt#105] +Arguments: [ss_ticket_number#90, ss_item_sk#89], [sr_ticket_number#103, sr_item_sk#102], LeftOuter -(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] +(101) CometProject 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] +Arguments: [d_year#101, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99, sales_cnt#106, sales_amt#107], [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] -(105) Scan parquet spark_catalog.default.web_sales +(102) 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 [] @@ -608,184 +594,194 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#112), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(106) CometFilter +(103) 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) -(107) ReusedExchange [Reuses operator id: 6] +(104) 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] -(108) CometBroadcastHashJoin +(105) 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 -(109) CometProject +(106) 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] -(110) ReusedExchange [Reuses operator id: 80] +(107) ReusedExchange [Reuses operator id: 79] Output [2]: [d_date_sk#119, d_year#120] -(111) CometBroadcastHashJoin +(108) 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 -(112) CometProject +(109) 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] - -(114) Exchange +(110) CometColumnarExchange 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=12] +Arguments: hashpartitioning(ws_order_number#109, ws_item_sk#108, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(115) Sort [codegen id : 31] +(111) CometSort 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 +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_order_number#109 ASC NULLS FIRST, ws_item_sk#108 ASC NULLS FIRST] -(116) ReusedExchange [Reuses operator id: 59] +(112) ReusedExchange [Reuses operator id: 53] Output [4]: [wr_item_sk#121, wr_order_number#122, wr_return_quantity#123, wr_return_amt#124] -(117) Sort [codegen id : 33] +(113) CometSort 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 +Arguments: [wr_item_sk#121, wr_order_number#122, wr_return_quantity#123, wr_return_amt#124], [wr_order_number#122 ASC NULLS FIRST, wr_item_sk#121 ASC NULLS FIRST] -(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 +(114) CometSortMergeJoin +Left 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] +Right output [4]: [wr_item_sk#121, wr_order_number#122, wr_return_quantity#123, wr_return_amt#124] +Arguments: [ws_order_number#109, ws_item_sk#108], [wr_order_number#122, wr_item_sk#121], LeftOuter -(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] +(115) CometProject 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] +Arguments: [d_year#120, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118, sales_cnt#125, sales_amt#126], [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] -(120) Union +(116) CometUnion +Child 0 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] +Child 1 Input [7]: [d_year#101, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99, sales_cnt#106, sales_amt#107] +Child 2 Input [7]: [d_year#120, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118, sales_cnt#125, sales_amt#126] -(121) HashAggregate [codegen id : 35] +(117) CometHashAggregate 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] -(122) Exchange +(118) CometColumnarExchange 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=13] +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, CometColumnarShuffle, [plan_id=13] -(123) HashAggregate [codegen id : 36] +(119) CometHashAggregate 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] -(124) HashAggregate [codegen id : 36] +(120) ColumnarToRow [codegen id : 4] +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] + +(121) HashAggregate [codegen id : 4] 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] -(125) Exchange +(122) RowToColumnar +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] + +(123) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=14] + +(124) ColumnarToRow [codegen id : 5] 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=14] -(126) HashAggregate [codegen id : 37] +(125) HashAggregate [codegen id : 5] 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] -(127) Filter [codegen id : 37] +(126) Filter [codegen id : 5] 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) -(128) Exchange +(127) RowToColumnar +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] + +(128) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=15] + +(129) CometSort 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=15] +Arguments: [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], [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] -(129) Sort [codegen id : 38] +(130) ColumnarToRow [codegen id : 6] 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 -(130) SortMergeJoin [codegen id : 39] +(131) SortMergeJoin [codegen id : 7] 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) -(131) Project [codegen id : 39] +(132) Project [codegen id : 7] 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] -(132) TakeOrderedAndProject +(133) 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 (136) -+- * ColumnarToRow (135) - +- CometFilter (134) - +- CometScan parquet spark_catalog.default.date_dim (133) +BroadcastExchange (137) ++- * ColumnarToRow (136) + +- CometFilter (135) + +- CometScan parquet spark_catalog.default.date_dim (134) -(133) Scan parquet spark_catalog.default.date_dim +(134) 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 -(134) CometFilter +(135) 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)) -(135) ColumnarToRow [codegen id : 1] +(136) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] -(136) BroadcastExchange +(137) BroadcastExchange Input [2]: [d_date_sk#13, d_year#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 -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) +Subquery:4 Hosting operator id = 72 Hosting Expression = cs_sold_date_sk#74 IN dynamicpruning#75 +BroadcastExchange (141) ++- * ColumnarToRow (140) + +- CometFilter (139) + +- CometScan parquet spark_catalog.default.date_dim (138) -(137) Scan parquet spark_catalog.default.date_dim +(138) 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 -(138) CometFilter +(139) 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)) -(139) ColumnarToRow [codegen id : 1] +(140) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#81, d_year#82] -(140) BroadcastExchange +(141) BroadcastExchange Input [2]: [d_date_sk#81, d_year#82] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] -Subquery:5 Hosting operator id = 90 Hosting Expression = ss_sold_date_sk#93 IN dynamicpruning#75 +Subquery:5 Hosting operator id = 88 Hosting Expression = ss_sold_date_sk#93 IN dynamicpruning#75 -Subquery:6 Hosting operator id = 105 Hosting Expression = ws_sold_date_sk#112 IN dynamicpruning#75 +Subquery:6 Hosting operator id = 102 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 27a4dcb67..3d82352b7 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,229 +1,166 @@ 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 (39) + WholeStageCodegen (7) 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 (19) - Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] + WholeStageCodegen (3) + ColumnarToRow InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - 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 (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 (16) - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 + RowToColumnar + WholeStageCodegen (2) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] + ColumnarToRow InputAdapter - Union - 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 (2) - Sort [cs_order_number,cs_item_sk] - InputAdapter - Exchange [cs_order_number,cs_item_sk] #4 - 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_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,d_date_sk,d_year] - 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,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_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 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,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 [d_date_sk,d_year] #7 - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - WholeStageCodegen (4) - Sort [cr_order_number,cr_item_sk] - InputAdapter - 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_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_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 (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 (7) - Sort [ss_ticket_number,ss_item_sk] - 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_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,d_date_sk,d_year] - 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,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_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 - 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 (9) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - 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_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_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 (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 (12) - Sort [ws_order_number,ws_item_sk] - 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_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,d_date_sk,d_year] - 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,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_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 - 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 (14) - Sort [wr_order_number,wr_item_sk] - InputAdapter - 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_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometSort [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] + CometColumnarExchange [cs_order_number,cs_item_sk] #4 + 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_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,d_date_sk,d_year] + 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,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_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 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,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 [d_date_sk,d_year] #7 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometColumnarExchange [cr_order_number,cr_item_sk] #8 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometSort [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] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #9 + 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_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,d_date_sk,d_year] + 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,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_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 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #7 + CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometColumnarExchange [sr_ticket_number,sr_item_sk] #10 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometSort [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] + CometColumnarExchange [ws_order_number,ws_item_sk] #11 + 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_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,d_date_sk,d_year] + 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,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_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 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #7 + CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometColumnarExchange [wr_order_number,wr_item_sk] #12 + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_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 (38) - Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] + WholeStageCodegen (6) + ColumnarToRow InputAdapter - 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] #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] #15 - WholeStageCodegen (35) - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 + RowToColumnar + WholeStageCodegen (5) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] + ColumnarToRow InputAdapter - Union - 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 (21) - Sort [cs_order_number,cs_item_sk] - 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_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,d_date_sk,d_year] - 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,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_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 #17 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometBroadcastExchange [d_date_sk,d_year] #18 - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - WholeStageCodegen (23) - Sort [cr_order_number,cr_item_sk] - InputAdapter - 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 (26) - Sort [ss_ticket_number,ss_item_sk] - 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_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,d_date_sk,d_year] - 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,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_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 - 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 (28) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - 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 (31) - Sort [ws_order_number,ws_item_sk] - 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_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,d_date_sk,d_year] - 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,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_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 - 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 (33) - Sort [wr_order_number,wr_item_sk] - InputAdapter - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #12 + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometSort [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] + CometColumnarExchange [cs_order_number,cs_item_sk] #16 + 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_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,d_date_sk,d_year] + 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,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_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 #17 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + CometBroadcastExchange [d_date_sk,d_year] #18 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #8 + CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometSort [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] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #19 + 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_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,d_date_sk,d_year] + 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,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_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 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #18 + CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #10 + CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometSort [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] + CometColumnarExchange [ws_order_number,ws_item_sk] #20 + 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_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,d_date_sk,d_year] + 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,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_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 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #18 + CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + 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 088855406..8aa8b1c8c 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,38 +1,40 @@ == Physical Plan == -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) +TakeOrderedAndProject (36) ++- * HashAggregate (35) + +- * ColumnarToRow (34) + +- CometColumnarExchange (33) + +- RowToColumnar (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 @@ -182,18 +184,24 @@ 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] -(32) Exchange +(32) RowToColumnar 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=1] -(33) HashAggregate [codegen id : 2] +(33) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(34) ColumnarToRow [codegen id : 2] +Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count#39, sum#40] + +(35) 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] -(34) TakeOrderedAndProject +(36) 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 cfcf6a99a..66700151a 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,38 +1,40 @@ TakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] 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 (1) - HashAggregate [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] [count,sum,count,sum] - ColumnarToRow - InputAdapter - CometUnion [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometProject [ss_store_sk,ss_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] - CometProject [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_category] - CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_category] #2 - CometFilter [i_item_sk,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_category] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #3 - CometFilter [d_date_sk,d_year,d_qoy] - 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_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] - CometProject [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_category] - CometFilter [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_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_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] - CometProject [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category] - CometBroadcastHashJoin [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_category] - CometFilter [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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 + ColumnarToRow + InputAdapter + CometColumnarExchange [channel,col_name,d_year,d_qoy,i_category] #1 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] [count,sum,count,sum] + ColumnarToRow + InputAdapter + CometUnion [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] + CometProject [ss_store_sk,ss_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] + CometProject [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_category] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange [i_item_sk,i_category] #2 + CometFilter [i_item_sk,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_category] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #3 + CometFilter [d_date_sk,d_year,d_qoy] + 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_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] + CometProject [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_category] + CometFilter [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_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_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] + CometProject [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category] + CometBroadcastHashJoin [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_category] + CometFilter [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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 c177fb453..7acc17e5f 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,90 +1,104 @@ == Physical Plan == -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) +TakeOrderedAndProject (100) ++- * HashAggregate (99) + +- * ColumnarToRow (98) + +- CometColumnarExchange (97) + +- RowToColumnar (96) + +- * HashAggregate (95) + +- * Expand (94) + +- Union (93) + :- * Project (36) + : +- * BroadcastHashJoin LeftOuter BuildRight (35) + : :- * HashAggregate (19) + : : +- * ColumnarToRow (18) + : : +- CometColumnarExchange (17) + : : +- RowToColumnar (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 (34) + : +- * HashAggregate (33) + : +- * ColumnarToRow (32) + : +- CometColumnarExchange (31) + : +- RowToColumnar (30) + : +- * HashAggregate (29) + : +- * ColumnarToRow (28) + : +- CometProject (27) + : +- CometBroadcastHashJoin (26) + : :- CometProject (24) + : : +- CometBroadcastHashJoin (23) + : : :- CometFilter (21) + : : : +- CometScan parquet spark_catalog.default.store_returns (20) + : : +- ReusedExchange (22) + : +- ReusedExchange (25) + :- * Project (59) + : +- * BroadcastNestedLoopJoin Inner BuildLeft (58) + : :- BroadcastExchange (47) + : : +- * HashAggregate (46) + : : +- * ColumnarToRow (45) + : : +- CometColumnarExchange (44) + : : +- RowToColumnar (43) + : : +- * HashAggregate (42) + : : +- * ColumnarToRow (41) + : : +- CometProject (40) + : : +- CometBroadcastHashJoin (39) + : : :- CometScan parquet spark_catalog.default.catalog_sales (37) + : : +- ReusedExchange (38) + : +- * HashAggregate (57) + : +- * ColumnarToRow (56) + : +- CometColumnarExchange (55) + : +- RowToColumnar (54) + : +- * HashAggregate (53) + : +- * ColumnarToRow (52) + : +- CometProject (51) + : +- CometBroadcastHashJoin (50) + : :- CometScan parquet spark_catalog.default.catalog_returns (48) + : +- ReusedExchange (49) + +- * Project (92) + +- * BroadcastHashJoin LeftOuter BuildRight (91) + :- * HashAggregate (75) + : +- * ColumnarToRow (74) + : +- CometColumnarExchange (73) + : +- RowToColumnar (72) + : +- * HashAggregate (71) + : +- * ColumnarToRow (70) + : +- CometProject (69) + : +- CometBroadcastHashJoin (68) + : :- CometProject (64) + : : +- CometBroadcastHashJoin (63) + : : :- CometFilter (61) + : : : +- CometScan parquet spark_catalog.default.web_sales (60) + : : +- ReusedExchange (62) + : +- CometBroadcastExchange (67) + : +- CometFilter (66) + : +- CometScan parquet spark_catalog.default.web_page (65) + +- BroadcastExchange (90) + +- * HashAggregate (89) + +- * ColumnarToRow (88) + +- CometColumnarExchange (87) + +- RowToColumnar (86) + +- * HashAggregate (85) + +- * ColumnarToRow (84) + +- CometProject (83) + +- CometBroadcastHashJoin (82) + :- CometProject (80) + : +- CometBroadcastHashJoin (79) + : :- CometFilter (77) + : : +- CometScan parquet spark_catalog.default.web_returns (76) + : +- ReusedExchange (78) + +- ReusedExchange (81) (1) Scan parquet spark_catalog.default.store_sales @@ -161,18 +175,24 @@ Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(Un Aggregate Attributes [2]: [sum#9, sum#10] Results [3]: [s_store_sk#8, sum#11, sum#12] -(16) Exchange +(16) RowToColumnar Input [3]: [s_store_sk#8, sum#11, sum#12] -Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(17) HashAggregate [codegen id : 4] +(17) CometColumnarExchange +Input [3]: [s_store_sk#8, sum#11, sum#12] +Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(18) ColumnarToRow [codegen id : 4] +Input [3]: [s_store_sk#8, sum#11, sum#12] + +(19) 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))#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] -(18) Scan parquet spark_catalog.default.store_returns +(20) 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 [] @@ -180,162 +200,180 @@ PartitionFilters: [isnotnull(sr_returned_date_sk#20), dynamicpruningexpression(s PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(19) CometFilter +(21) 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] +(22) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#22] -(21) CometBroadcastHashJoin +(23) 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 +(24) 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] +(25) ReusedExchange [Reuses operator id: 11] Output [1]: [s_store_sk#23] -(24) CometBroadcastHashJoin +(26) 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 +(27) 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] +(28) ColumnarToRow [codegen id : 2] Input [3]: [sr_return_amt#18, sr_net_loss#19, s_store_sk#23] -(27) HashAggregate [codegen id : 2] +(29) 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 +(30) RowToColumnar +Input [3]: [s_store_sk#23, sum#26, sum#27] + +(31) CometColumnarExchange +Input [3]: [s_store_sk#23, sum#26, sum#27] +Arguments: hashpartitioning(s_store_sk#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(32) ColumnarToRow [codegen id : 3] 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] +(33) 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 +(34) 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] +(35) BroadcastHashJoin [codegen id : 4] Left keys [1]: [s_store_sk#8] Right keys [1]: [s_store_sk#23] Join type: LeftOuter Join condition: None -(32) Project [codegen id : 4] +(36) 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] -(33) Scan parquet spark_catalog.default.catalog_sales +(37) 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#39), dynamicpruningexpression(cs_sold_date_sk#39 IN dynamicpruning#40)] ReadSchema: struct -(34) ReusedExchange [Reuses operator id: 6] +(38) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#41] -(35) CometBroadcastHashJoin +(39) 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 +(40) 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] +(41) 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] +(42) 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 +(43) RowToColumnar +Input [3]: [cs_call_center_sk#36, sum#44, sum#45] + +(44) CometColumnarExchange Input [3]: [cs_call_center_sk#36, sum#44, sum#45] -Arguments: hashpartitioning(cs_call_center_sk#36, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: hashpartitioning(cs_call_center_sk#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(40) HashAggregate [codegen id : 6] +(45) ColumnarToRow [codegen id : 6] +Input [3]: [cs_call_center_sk#36, sum#44, sum#45] + +(46) 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 +(47) 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 +(48) 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#52), dynamicpruningexpression(cr_returned_date_sk#52 IN dynamicpruning#53)] ReadSchema: struct -(43) ReusedExchange [Reuses operator id: 6] +(49) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#54] -(44) CometBroadcastHashJoin +(50) 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 -(45) CometProject +(51) 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] -(46) ColumnarToRow [codegen id : 7] +(52) ColumnarToRow [codegen id : 7] Input [2]: [cr_return_amount#50, cr_net_loss#51] -(47) HashAggregate [codegen id : 7] +(53) HashAggregate [codegen id : 7] Input [2]: [cr_return_amount#50, cr_net_loss#51] Keys: [] 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] -(48) Exchange +(54) RowToColumnar +Input [2]: [sum#57, sum#58] + +(55) CometColumnarExchange Input [2]: [sum#57, sum#58] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(49) HashAggregate +(56) ColumnarToRow +Input [2]: [sum#57, sum#58] + +(57) HashAggregate Input [2]: [sum#57, sum#58] Keys: [] 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] -(50) BroadcastNestedLoopJoin [codegen id : 8] +(58) BroadcastNestedLoopJoin [codegen id : 8] Join type: Inner Join condition: None -(51) Project [codegen id : 8] +(59) 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] -(52) Scan parquet spark_catalog.default.web_sales +(60) 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 [] @@ -343,68 +381,74 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#69), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct -(53) CometFilter +(61) 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) -(54) ReusedExchange [Reuses operator id: 6] +(62) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#71] -(55) CometBroadcastHashJoin +(63) 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 -(56) CometProject +(64) 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] -(57) Scan parquet spark_catalog.default.web_page +(65) 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 -(58) CometFilter +(66) CometFilter Input [1]: [wp_web_page_sk#72] Condition : isnotnull(wp_web_page_sk#72) -(59) CometBroadcastExchange +(67) CometBroadcastExchange Input [1]: [wp_web_page_sk#72] Arguments: [wp_web_page_sk#72] -(60) CometBroadcastHashJoin +(68) 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 +(69) 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] +(70) 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] +(71) 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 +(72) RowToColumnar +Input [3]: [wp_web_page_sk#72, sum#75, sum#76] + +(73) CometColumnarExchange Input [3]: [wp_web_page_sk#72, sum#75, sum#76] -Arguments: hashpartitioning(wp_web_page_sk#72, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Arguments: hashpartitioning(wp_web_page_sk#72, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(65) HashAggregate [codegen id : 12] +(74) ColumnarToRow [codegen id : 12] +Input [3]: [wp_web_page_sk#72, sum#75, sum#76] + +(75) 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 +(76) 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 [] @@ -412,137 +456,149 @@ PartitionFilters: [isnotnull(wr_returned_date_sk#84), dynamicpruningexpression(w PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct -(67) CometFilter +(77) 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: 6] +(78) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#86] -(69) CometBroadcastHashJoin +(79) 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) CometProject +(80) 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] +(81) ReusedExchange [Reuses operator id: 67] Output [1]: [wp_web_page_sk#87] -(72) CometBroadcastHashJoin +(82) 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 +(83) 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] +(84) ColumnarToRow [codegen id : 10] Input [3]: [wr_return_amt#82, wr_net_loss#83, wp_web_page_sk#87] -(75) HashAggregate [codegen id : 10] +(85) 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 +(86) RowToColumnar +Input [3]: [wp_web_page_sk#87, sum#90, sum#91] + +(87) CometColumnarExchange +Input [3]: [wp_web_page_sk#87, sum#90, sum#91] +Arguments: hashpartitioning(wp_web_page_sk#87, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(88) ColumnarToRow [codegen id : 11] 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] +(89) 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 +(90) 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] +(91) 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 -(80) Project [codegen id : 12] +(92) 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] -(81) Union +(93) Union -(82) Expand [codegen id : 13] +(94) 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] -(83) HashAggregate [codegen id : 13] +(95) 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] -(84) Exchange +(96) RowToColumnar +Input [9]: [channel#100, id#101, spark_grouping_id#102, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] + +(97) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=10] + +(98) ColumnarToRow [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] -Arguments: hashpartitioning(channel#100, id#101, spark_grouping_id#102, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(85) HashAggregate [codegen id : 14] +(99) 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] -(86) TakeOrderedAndProject +(100) 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 (91) -+- * ColumnarToRow (90) - +- CometProject (89) - +- CometFilter (88) - +- CometScan parquet spark_catalog.default.date_dim (87) +BroadcastExchange (105) ++- * ColumnarToRow (104) + +- CometProject (103) + +- CometFilter (102) + +- CometScan parquet spark_catalog.default.date_dim (101) -(87) Scan parquet spark_catalog.default.date_dim +(101) 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 -(88) CometFilter +(102) 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)) -(89) CometProject +(103) CometProject Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(90) ColumnarToRow [codegen id : 1] +(104) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(91) BroadcastExchange +(105) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -Subquery:2 Hosting operator id = 18 Hosting Expression = sr_returned_date_sk#20 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 20 Hosting Expression = sr_returned_date_sk#20 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#39 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 37 Hosting Expression = cs_sold_date_sk#39 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 42 Hosting Expression = cr_returned_date_sk#52 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 48 Hosting Expression = cr_returned_date_sk#52 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 52 Hosting Expression = ws_sold_date_sk#69 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#69 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 66 Hosting Expression = wr_returned_date_sk#84 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 76 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 91786374e..b5b538743 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,130 +1,144 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] 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 (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 (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 (1) - HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] - ColumnarToRow - InputAdapter - CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] - CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk] #5 - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] - InputAdapter - 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel,id,spark_grouping_id] #1 + RowToColumnar + 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 (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] + ColumnarToRow InputAdapter - Exchange [s_store_sk] #7 - WholeStageCodegen (2) - HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] - ColumnarToRow - InputAdapter - CometProject [sr_return_amt,sr_net_loss,s_store_sk] - CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] - CometProject [sr_store_sk,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_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 - 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 #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] + CometColumnarExchange [s_store_sk] #2 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] + ColumnarToRow + InputAdapter + CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [s_store_sk] #5 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] + InputAdapter + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [s_store_sk] #7 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] + ColumnarToRow + InputAdapter + CometProject [sr_return_amt,sr_net_loss,s_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] + CometProject [sr_store_sk,sr_return_amt,sr_net_loss] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_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 + 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 #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] + ColumnarToRow + InputAdapter + CometColumnarExchange [cs_call_center_sk] #9 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] + ColumnarToRow + InputAdapter + CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,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 + ReusedExchange [d_date_sk] #4 + HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] + ColumnarToRow InputAdapter - Exchange [cs_call_center_sk] #9 - WholeStageCodegen (5) - HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] - ColumnarToRow - InputAdapter - CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,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 - 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 #10 - WholeStageCodegen (7) - HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] - ColumnarToRow - InputAdapter - CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_return_amount,cr_net_loss,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 - 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] #11 - WholeStageCodegen (9) - HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] - ColumnarToRow - InputAdapter - CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk] - CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_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 - ReusedExchange [d_date_sk] #4 - CometBroadcastExchange [wp_web_page_sk] #12 - CometFilter [wp_web_page_sk] - CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] - InputAdapter - 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] + CometColumnarExchange #10 + RowToColumnar + WholeStageCodegen (7) + HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] + ColumnarToRow + InputAdapter + CometProject [cr_return_amount,cr_net_loss] + CometBroadcastHashJoin [cr_return_amount,cr_net_loss,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 + 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] + ColumnarToRow InputAdapter - Exchange [wp_web_page_sk] #14 - WholeStageCodegen (10) - HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] - ColumnarToRow - InputAdapter - CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] - CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] - CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk] - CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_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 - ReusedExchange [d_date_sk] #4 - ReusedExchange [wp_web_page_sk] #12 + CometColumnarExchange [wp_web_page_sk] #11 + RowToColumnar + WholeStageCodegen (9) + HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] + ColumnarToRow + InputAdapter + CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] + CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk] + CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_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 + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange [wp_web_page_sk] #12 + CometFilter [wp_web_page_sk] + CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] + InputAdapter + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [wp_web_page_sk] #14 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] + ColumnarToRow + InputAdapter + CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] + CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk] + CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_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 + 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/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/explain.txt index 7f2688112..f2886073f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/explain.txt @@ -1,74 +1,79 @@ == Physical Plan == -TakeOrderedAndProject (70) -+- * Project (69) - +- * SortMergeJoin Inner (68) - :- * Project (45) - : +- * SortMergeJoin Inner (44) - : :- * Sort (21) - : : +- * HashAggregate (20) - : : +- Exchange (19) - : : +- * HashAggregate (18) - : : +- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Project (14) - : : : +- * Filter (13) - : : : +- * SortMergeJoin LeftOuter (12) - : : : :- * Sort (5) - : : : : +- Exchange (4) - : : : : +- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- * Sort (11) - : : : +- Exchange (10) - : : : +- * ColumnarToRow (9) - : : : +- CometProject (8) - : : : +- CometFilter (7) - : : : +- CometScan parquet spark_catalog.default.store_returns (6) - : : +- ReusedExchange (15) - : +- * Sort (43) - : +- * Filter (42) - : +- * HashAggregate (41) - : +- Exchange (40) - : +- * HashAggregate (39) - : +- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * Project (35) - : : +- * Filter (34) - : : +- * SortMergeJoin LeftOuter (33) - : : :- * Sort (26) - : : : +- Exchange (25) - : : : +- * ColumnarToRow (24) - : : : +- CometFilter (23) - : : : +- CometScan parquet spark_catalog.default.web_sales (22) - : : +- * Sort (32) - : : +- Exchange (31) - : : +- * ColumnarToRow (30) - : : +- CometProject (29) - : : +- CometFilter (28) - : : +- CometScan parquet spark_catalog.default.web_returns (27) - : +- ReusedExchange (36) - +- * Sort (67) - +- * Filter (66) - +- * HashAggregate (65) - +- Exchange (64) - +- * HashAggregate (63) - +- * Project (62) - +- * BroadcastHashJoin Inner BuildRight (61) - :- * Project (59) - : +- * Filter (58) - : +- * SortMergeJoin LeftOuter (57) - : :- * Sort (50) - : : +- Exchange (49) - : : +- * ColumnarToRow (48) - : : +- CometFilter (47) - : : +- CometScan parquet spark_catalog.default.catalog_sales (46) - : +- * Sort (56) - : +- Exchange (55) - : +- * ColumnarToRow (54) - : +- CometProject (53) - : +- CometFilter (52) - : +- CometScan parquet spark_catalog.default.catalog_returns (51) - +- ReusedExchange (60) +TakeOrderedAndProject (75) ++- * Project (74) + +- * SortMergeJoin Inner (73) + :- * Project (49) + : +- * SortMergeJoin Inner (48) + : :- * Sort (24) + : : +- * HashAggregate (23) + : : +- * ColumnarToRow (22) + : : +- CometColumnarExchange (21) + : : +- RowToColumnar (20) + : : +- * HashAggregate (19) + : : +- * ColumnarToRow (18) + : : +- CometProject (17) + : : +- CometBroadcastHashJoin (16) + : : :- CometProject (12) + : : : +- CometFilter (11) + : : : +- CometSortMergeJoin (10) + : : : :- CometSort (4) + : : : : +- CometColumnarExchange (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometSort (9) + : : : +- CometColumnarExchange (8) + : : : +- CometProject (7) + : : : +- CometFilter (6) + : : : +- CometScan parquet spark_catalog.default.store_returns (5) + : : +- CometBroadcastExchange (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.date_dim (13) + : +- * Sort (47) + : +- * Filter (46) + : +- * HashAggregate (45) + : +- * ColumnarToRow (44) + : +- CometColumnarExchange (43) + : +- RowToColumnar (42) + : +- * HashAggregate (41) + : +- * ColumnarToRow (40) + : +- CometProject (39) + : +- CometBroadcastHashJoin (38) + : :- CometProject (36) + : : +- CometFilter (35) + : : +- CometSortMergeJoin (34) + : : :- CometSort (28) + : : : +- CometColumnarExchange (27) + : : : +- CometFilter (26) + : : : +- CometScan parquet spark_catalog.default.web_sales (25) + : : +- CometSort (33) + : : +- CometColumnarExchange (32) + : : +- CometProject (31) + : : +- CometFilter (30) + : : +- CometScan parquet spark_catalog.default.web_returns (29) + : +- ReusedExchange (37) + +- * Sort (72) + +- * Filter (71) + +- * HashAggregate (70) + +- * ColumnarToRow (69) + +- CometColumnarExchange (68) + +- RowToColumnar (67) + +- * HashAggregate (66) + +- * ColumnarToRow (65) + +- CometProject (64) + +- CometBroadcastHashJoin (63) + :- CometProject (61) + : +- CometFilter (60) + : +- CometSortMergeJoin (59) + : :- CometSort (53) + : : +- CometColumnarExchange (52) + : : +- CometFilter (51) + : : +- CometScan parquet spark_catalog.default.catalog_sales (50) + : +- CometSort (58) + : +- CometColumnarExchange (57) + : +- CometProject (56) + : +- CometFilter (55) + : +- CometScan parquet spark_catalog.default.catalog_returns (54) + +- ReusedExchange (62) (1) Scan parquet spark_catalog.default.store_sales @@ -83,93 +88,106 @@ ReadSchema: struct -(7) CometFilter +(6) CometFilter Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] Condition : (isnotnull(sr_ticket_number#10) AND isnotnull(sr_item_sk#9)) -(8) CometProject +(7) CometProject Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_item_sk#9, sr_ticket_number#10] -(9) ColumnarToRow [codegen id : 3] -Input [2]: [sr_item_sk#9, sr_ticket_number#10] - -(10) Exchange +(8) CometColumnarExchange Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(11) Sort [codegen id : 4] +(9) CometSort Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST], false, 0 +Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST] -(12) SortMergeJoin [codegen id : 6] -Left keys [2]: [ss_ticket_number#3, ss_item_sk#1] -Right keys [2]: [sr_ticket_number#10, sr_item_sk#9] -Join type: LeftOuter -Join condition: None +(10) CometSortMergeJoin +Left output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] +Right output [2]: [sr_item_sk#9, sr_ticket_number#10] +Arguments: [ss_ticket_number#3, ss_item_sk#1], [sr_ticket_number#10, sr_item_sk#9], LeftOuter -(13) Filter [codegen id : 6] +(11) CometFilter Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] Condition : isnull(sr_ticket_number#10) -(14) Project [codegen id : 6] -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] +(12) CometProject Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -(15) ReusedExchange [Reuses operator id: 74] +(13) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#12, d_year#13] +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 -(16) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None +(14) CometFilter +Input [2]: [d_date_sk#12, d_year#13] +Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) + +(15) CometBroadcastExchange +Input [2]: [d_date_sk#12, d_year#13] +Arguments: [d_date_sk#12, d_year#13] -(17) Project [codegen id : 6] -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] +(16) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] +Right output [2]: [d_date_sk#12, d_year#13] +Arguments: [ss_sold_date_sk#7], [d_date_sk#12], Inner, BuildRight + +(17) CometProject Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] + +(18) ColumnarToRow [codegen id : 1] +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] -(18) HashAggregate [codegen id : 6] +(19) HashAggregate [codegen id : 1] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesale_cost#5)), partial_sum(UnscaledValue(ss_sales_price#6))] Aggregate Attributes [3]: [sum#14, sum#15, sum#16] Results [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] -(19) Exchange +(20) RowToColumnar Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] -Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(20) HashAggregate [codegen id : 7] +(21) CometColumnarExchange +Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] +Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(22) ColumnarToRow [codegen id : 2] +Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] + +(23) HashAggregate [codegen id : 2] Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] Aggregate Attributes [3]: [sum(ss_quantity#4)#20, sum(UnscaledValue(ss_wholesale_cost#5))#21, sum(UnscaledValue(ss_sales_price#6))#22] Results [6]: [d_year#13 AS ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, sum(ss_quantity#4)#20 AS ss_qty#24, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#5))#21,17,2) AS ss_wc#25, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#22,17,2) AS ss_sp#26] -(21) Sort [codegen id : 7] +(24) Sort [codegen id : 2] Input [6]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] Arguments: [ss_sold_year#23 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], false, 0 -(22) Scan parquet spark_catalog.default.web_sales +(25) Scan parquet spark_catalog.default.web_sales Output [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] @@ -177,111 +195,112 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(23) CometFilter +(26) CometFilter Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] Condition : (isnotnull(ws_item_sk#27) AND isnotnull(ws_bill_customer_sk#28)) -(24) ColumnarToRow [codegen id : 8] -Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] - -(25) Exchange +(27) CometColumnarExchange Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Arguments: hashpartitioning(ws_order_number#29, ws_item_sk#27, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: hashpartitioning(ws_order_number#29, ws_item_sk#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(26) Sort [codegen id : 9] +(28) CometSort Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Arguments: [ws_order_number#29 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST], false, 0 +Arguments: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33], [ws_order_number#29 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST] -(27) Scan parquet spark_catalog.default.web_returns +(29) Scan parquet spark_catalog.default.web_returns Output [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct -(28) CometFilter +(30) CometFilter Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] Condition : (isnotnull(wr_order_number#36) AND isnotnull(wr_item_sk#35)) -(29) CometProject +(31) CometProject Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] Arguments: [wr_item_sk#35, wr_order_number#36], [wr_item_sk#35, wr_order_number#36] -(30) ColumnarToRow [codegen id : 10] +(32) CometColumnarExchange Input [2]: [wr_item_sk#35, wr_order_number#36] +Arguments: hashpartitioning(wr_order_number#36, wr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(31) Exchange +(33) CometSort Input [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: hashpartitioning(wr_order_number#36, wr_item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Arguments: [wr_item_sk#35, wr_order_number#36], [wr_order_number#36 ASC NULLS FIRST, wr_item_sk#35 ASC NULLS FIRST] -(32) Sort [codegen id : 11] -Input [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: [wr_order_number#36 ASC NULLS FIRST, wr_item_sk#35 ASC NULLS FIRST], false, 0 - -(33) SortMergeJoin [codegen id : 13] -Left keys [2]: [ws_order_number#29, ws_item_sk#27] -Right keys [2]: [wr_order_number#36, wr_item_sk#35] -Join type: LeftOuter -Join condition: None +(34) CometSortMergeJoin +Left output [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Right output [2]: [wr_item_sk#35, wr_order_number#36] +Arguments: [ws_order_number#29, ws_item_sk#27], [wr_order_number#36, wr_item_sk#35], LeftOuter -(34) Filter [codegen id : 13] +(35) CometFilter Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] Condition : isnull(wr_order_number#36) -(35) Project [codegen id : 13] -Output [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +(36) CometProject Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] +Arguments: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33], [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -(36) ReusedExchange [Reuses operator id: 74] +(37) ReusedExchange [Reuses operator id: 15] Output [2]: [d_date_sk#38, d_year#39] -(37) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#38] -Join type: Inner -Join condition: None +(38) CometBroadcastHashJoin +Left output [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Right output [2]: [d_date_sk#38, d_year#39] +Arguments: [ws_sold_date_sk#33], [d_date_sk#38], Inner, BuildRight -(38) Project [codegen id : 13] -Output [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] +(39) CometProject Input [8]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, d_date_sk#38, d_year#39] +Arguments: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39], [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] + +(40) ColumnarToRow [codegen id : 3] +Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] -(39) HashAggregate [codegen id : 13] +(41) HashAggregate [codegen id : 3] Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] Functions [3]: [partial_sum(ws_quantity#30), partial_sum(UnscaledValue(ws_wholesale_cost#31)), partial_sum(UnscaledValue(ws_sales_price#32))] Aggregate Attributes [3]: [sum#40, sum#41, sum#42] Results [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] -(40) Exchange +(42) RowToColumnar +Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] + +(43) CometColumnarExchange Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] -Arguments: hashpartitioning(d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Arguments: hashpartitioning(d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(41) HashAggregate [codegen id : 14] +(44) ColumnarToRow [codegen id : 4] +Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] + +(45) HashAggregate [codegen id : 4] Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] Functions [3]: [sum(ws_quantity#30), sum(UnscaledValue(ws_wholesale_cost#31)), sum(UnscaledValue(ws_sales_price#32))] Aggregate Attributes [3]: [sum(ws_quantity#30)#46, sum(UnscaledValue(ws_wholesale_cost#31))#47, sum(UnscaledValue(ws_sales_price#32))#48] Results [6]: [d_year#39 AS ws_sold_year#49, ws_item_sk#27, ws_bill_customer_sk#28 AS ws_customer_sk#50, sum(ws_quantity#30)#46 AS ws_qty#51, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#31))#47,17,2) AS ws_wc#52, MakeDecimal(sum(UnscaledValue(ws_sales_price#32))#48,17,2) AS ws_sp#53] -(42) Filter [codegen id : 14] +(46) Filter [codegen id : 4] Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] Condition : (coalesce(ws_qty#51, 0) > 0) -(43) Sort [codegen id : 14] +(47) Sort [codegen id : 4] Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] Arguments: [ws_sold_year#49 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST, ws_customer_sk#50 ASC NULLS FIRST], false, 0 -(44) SortMergeJoin [codegen id : 15] +(48) SortMergeJoin [codegen id : 5] Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] Right keys [3]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50] Join type: Inner Join condition: None -(45) Project [codegen id : 15] +(49) Project [codegen id : 5] Output [9]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53] Input [12]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] -(46) Scan parquet spark_catalog.default.catalog_sales +(50) Scan parquet spark_catalog.default.catalog_sales Output [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] Batched: true Location: InMemoryFileIndex [] @@ -289,143 +308,144 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(47) CometFilter +(51) CometFilter Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] Condition : (isnotnull(cs_item_sk#55) AND isnotnull(cs_bill_customer_sk#54)) -(48) ColumnarToRow [codegen id : 16] -Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] - -(49) Exchange +(52) CometColumnarExchange Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Arguments: hashpartitioning(cs_order_number#56, cs_item_sk#55, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Arguments: hashpartitioning(cs_order_number#56, cs_item_sk#55, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(50) Sort [codegen id : 17] +(53) CometSort Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Arguments: [cs_order_number#56 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST], false, 0 +Arguments: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60], [cs_order_number#56 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST] -(51) Scan parquet spark_catalog.default.catalog_returns +(54) Scan parquet spark_catalog.default.catalog_returns Output [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(52) CometFilter +(55) CometFilter Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] Condition : (isnotnull(cr_order_number#63) AND isnotnull(cr_item_sk#62)) -(53) CometProject +(56) CometProject Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] Arguments: [cr_item_sk#62, cr_order_number#63], [cr_item_sk#62, cr_order_number#63] -(54) ColumnarToRow [codegen id : 18] -Input [2]: [cr_item_sk#62, cr_order_number#63] - -(55) Exchange +(57) CometColumnarExchange Input [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: hashpartitioning(cr_order_number#63, cr_item_sk#62, 5), ENSURE_REQUIREMENTS, [plan_id=8] +Arguments: hashpartitioning(cr_order_number#63, cr_item_sk#62, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(56) Sort [codegen id : 19] +(58) CometSort Input [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: [cr_order_number#63 ASC NULLS FIRST, cr_item_sk#62 ASC NULLS FIRST], false, 0 +Arguments: [cr_item_sk#62, cr_order_number#63], [cr_order_number#63 ASC NULLS FIRST, cr_item_sk#62 ASC NULLS FIRST] -(57) SortMergeJoin [codegen id : 21] -Left keys [2]: [cs_order_number#56, cs_item_sk#55] -Right keys [2]: [cr_order_number#63, cr_item_sk#62] -Join type: LeftOuter -Join condition: None +(59) CometSortMergeJoin +Left output [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +Right output [2]: [cr_item_sk#62, cr_order_number#63] +Arguments: [cs_order_number#56, cs_item_sk#55], [cr_order_number#63, cr_item_sk#62], LeftOuter -(58) Filter [codegen id : 21] +(60) CometFilter Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] Condition : isnull(cr_order_number#63) -(59) Project [codegen id : 21] -Output [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +(61) CometProject Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] +Arguments: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60], [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -(60) ReusedExchange [Reuses operator id: 74] +(62) ReusedExchange [Reuses operator id: 15] Output [2]: [d_date_sk#65, d_year#66] -(61) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [cs_sold_date_sk#60] -Right keys [1]: [d_date_sk#65] -Join type: Inner -Join condition: None +(63) CometBroadcastHashJoin +Left output [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +Right output [2]: [d_date_sk#65, d_year#66] +Arguments: [cs_sold_date_sk#60], [d_date_sk#65], Inner, BuildRight -(62) Project [codegen id : 21] -Output [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] +(64) CometProject Input [8]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, d_date_sk#65, d_year#66] +Arguments: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66], [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] -(63) HashAggregate [codegen id : 21] +(65) ColumnarToRow [codegen id : 6] +Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] + +(66) HashAggregate [codegen id : 6] Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] Functions [3]: [partial_sum(cs_quantity#57), partial_sum(UnscaledValue(cs_wholesale_cost#58)), partial_sum(UnscaledValue(cs_sales_price#59))] Aggregate Attributes [3]: [sum#67, sum#68, sum#69] Results [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] -(64) Exchange +(67) RowToColumnar +Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] + +(68) CometColumnarExchange +Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] +Arguments: hashpartitioning(d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(69) ColumnarToRow [codegen id : 7] Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] -Arguments: hashpartitioning(d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(65) HashAggregate [codegen id : 22] +(70) HashAggregate [codegen id : 7] Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] Functions [3]: [sum(cs_quantity#57), sum(UnscaledValue(cs_wholesale_cost#58)), sum(UnscaledValue(cs_sales_price#59))] Aggregate Attributes [3]: [sum(cs_quantity#57)#73, sum(UnscaledValue(cs_wholesale_cost#58))#74, sum(UnscaledValue(cs_sales_price#59))#75] Results [6]: [d_year#66 AS cs_sold_year#76, cs_item_sk#55, cs_bill_customer_sk#54 AS cs_customer_sk#77, sum(cs_quantity#57)#73 AS cs_qty#78, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#58))#74,17,2) AS cs_wc#79, MakeDecimal(sum(UnscaledValue(cs_sales_price#59))#75,17,2) AS cs_sp#80] -(66) Filter [codegen id : 22] +(71) Filter [codegen id : 7] Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] Condition : (coalesce(cs_qty#78, 0) > 0) -(67) Sort [codegen id : 22] +(72) Sort [codegen id : 7] Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] Arguments: [cs_sold_year#76 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST, cs_customer_sk#77 ASC NULLS FIRST], false, 0 -(68) SortMergeJoin [codegen id : 23] +(73) SortMergeJoin [codegen id : 8] Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] Right keys [3]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77] Join type: Inner Join condition: None -(69) Project [codegen id : 23] +(74) Project [codegen id : 8] Output [12]: [round((cast(ss_qty#24 as double) / cast(coalesce((ws_qty#51 + cs_qty#78), 1) as double)), 2) AS ratio#81, ss_qty#24 AS store_qty#82, ss_wc#25 AS store_wholesale_cost#83, ss_sp#26 AS store_sales_price#84, (coalesce(ws_qty#51, 0) + coalesce(cs_qty#78, 0)) AS other_chan_qty#85, (coalesce(ws_wc#52, 0.00) + coalesce(cs_wc#79, 0.00)) AS other_chan_wholesale_cost#86, (coalesce(ws_sp#53, 0.00) + coalesce(cs_sp#80, 0.00)) AS other_chan_sales_price#87, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, cs_qty#78] Input [15]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53, cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] -(70) TakeOrderedAndProject +(75) TakeOrderedAndProject Input [12]: [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, cs_qty#78] Arguments: 100, [ratio#81 ASC NULLS FIRST, ss_qty#24 DESC NULLS LAST, ss_wc#25 DESC NULLS LAST, ss_sp#26 DESC NULLS LAST, other_chan_qty#85 ASC NULLS FIRST, other_chan_wholesale_cost#86 ASC NULLS FIRST, other_chan_sales_price#87 ASC NULLS FIRST, round((cast(ss_qty#24 as double) / cast(coalesce((ws_qty#51 + cs_qty#78), 1) as double)), 2) ASC NULLS FIRST], [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (74) -+- * ColumnarToRow (73) - +- CometFilter (72) - +- CometScan parquet spark_catalog.default.date_dim (71) +BroadcastExchange (79) ++- * ColumnarToRow (78) + +- CometFilter (77) + +- CometScan parquet spark_catalog.default.date_dim (76) -(71) Scan parquet spark_catalog.default.date_dim +(76) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#12, d_year#13] 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 -(72) CometFilter +(77) CometFilter Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(73) ColumnarToRow [codegen id : 1] +(78) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] -(74) BroadcastExchange +(79) BroadcastExchange Input [2]: [d_date_sk#12, d_year#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 25 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 50 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/simplified.txt index 13a8aecff..1475eef5a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/simplified.txt @@ -1,127 +1,102 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ws_qty,cs_qty,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (23) + WholeStageCodegen (8) Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp] SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,cs_sold_year,cs_item_sk,cs_customer_sk] InputAdapter - WholeStageCodegen (15) + WholeStageCodegen (5) Project [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ws_sold_year,ws_item_sk,ws_customer_sk] InputAdapter - WholeStageCodegen (7) + WholeStageCodegen (2) Sort [ss_sold_year,ss_item_sk,ss_customer_sk] HashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] [sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price)),ss_sold_year,ss_qty,ss_wc,ss_sp,sum,sum,sum] - InputAdapter - Exchange [d_year,ss_item_sk,ss_customer_sk] #1 - WholeStageCodegen (6) - HashAggregate [d_year,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] [sum,sum,sum,sum,sum,sum] - Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - Filter [sr_ticket_number] - SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - WholeStageCodegen (2) - Sort [ss_ticket_number,ss_item_sk] - InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - WholeStageCodegen (4) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - Exchange [sr_ticket_number,sr_item_sk] #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + ColumnarToRow + InputAdapter + CometColumnarExchange [d_year,ss_item_sk,ss_customer_sk] #1 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [d_year,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] [sum,sum,sum,sum,sum,sum] + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #2 + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometSort [sr_item_sk,sr_ticket_number] + CometColumnarExchange [sr_ticket_number,sr_item_sk] #4 + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #5 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - WholeStageCodegen (14) + WholeStageCodegen (4) Sort [ws_sold_year,ws_item_sk,ws_customer_sk] Filter [ws_qty] HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] [sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price)),ws_sold_year,ws_customer_sk,ws_qty,ws_wc,ws_sp,sum,sum,sum] - InputAdapter - Exchange [d_year,ws_item_sk,ws_bill_customer_sk] #5 - WholeStageCodegen (13) - HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] [sum,sum,sum,sum,sum,sum] - Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - Filter [wr_order_number] - SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - InputAdapter - WholeStageCodegen (9) - Sort [ws_order_number,ws_item_sk] - InputAdapter - Exchange [ws_order_number,ws_item_sk] #6 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - WholeStageCodegen (11) - Sort [wr_order_number,wr_item_sk] - InputAdapter - Exchange [wr_order_number,wr_item_sk] #7 - WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + ColumnarToRow + InputAdapter + CometColumnarExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] [sum,sum,sum,sum,sum,sum] + ColumnarToRow + InputAdapter + CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] + CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,d_date_sk,d_year] + CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] + CometSortMergeJoin [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] + CometSort [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometColumnarExchange [ws_order_number,ws_item_sk] #7 + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_item_sk,wr_order_number] + CometColumnarExchange [wr_order_number,wr_item_sk] #8 + CometProject [wr_item_sk,wr_order_number] + CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] + ReusedExchange [d_date_sk,d_year] #5 InputAdapter - WholeStageCodegen (22) + WholeStageCodegen (7) Sort [cs_sold_year,cs_item_sk,cs_customer_sk] Filter [cs_qty] HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] [sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price)),cs_sold_year,cs_customer_sk,cs_qty,cs_wc,cs_sp,sum,sum,sum] - InputAdapter - Exchange [d_year,cs_item_sk,cs_bill_customer_sk] #8 - WholeStageCodegen (21) - HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] [sum,sum,sum,sum,sum,sum] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - Filter [cr_order_number] - SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - InputAdapter - WholeStageCodegen (17) - Sort [cs_order_number,cs_item_sk] - InputAdapter - Exchange [cs_order_number,cs_item_sk] #9 - WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - WholeStageCodegen (19) - Sort [cr_order_number,cr_item_sk] - InputAdapter - Exchange [cr_order_number,cr_item_sk] #10 - WholeStageCodegen (18) - ColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + ColumnarToRow + InputAdapter + CometColumnarExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] [sum,sum,sum,sum,sum,sum] + ColumnarToRow + InputAdapter + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] + CometSortMergeJoin [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] + CometSort [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + CometColumnarExchange [cs_order_number,cs_item_sk] #10 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [cr_item_sk,cr_order_number] + CometColumnarExchange [cr_order_number,cr_item_sk] #11 + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] + ReusedExchange [d_date_sk,d_year] #5 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 810eaa6e2..946c96906 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,35 +1,37 @@ == Physical Plan == -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) +TakeOrderedAndProject (33) ++- * Project (32) + +- * BroadcastHashJoin Inner BuildRight (31) + :- * HashAggregate (26) + : +- * ColumnarToRow (25) + : +- CometColumnarExchange (24) + : +- RowToColumnar (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 (30) + +- * ColumnarToRow (29) + +- CometFilter (28) + +- CometScan parquet spark_catalog.default.customer (27) (1) Scan parquet spark_catalog.default.store_sales @@ -138,78 +140,84 @@ Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(Unscale 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] -(23) Exchange +(23) RowToColumnar 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] -(24) HashAggregate [codegen id : 3] +(24) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(25) ColumnarToRow [codegen id : 3] +Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, sum#21, sum#22] + +(26) 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))#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] -(25) Scan parquet spark_catalog.default.customer +(27) 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 -(26) CometFilter +(28) CometFilter Input [3]: [c_customer_sk#27, c_first_name#28, c_last_name#29] Condition : isnotnull(c_customer_sk#27) -(27) ColumnarToRow [codegen id : 2] +(29) ColumnarToRow [codegen id : 2] Input [3]: [c_customer_sk#27, c_first_name#28, c_last_name#29] -(28) BroadcastExchange +(30) 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] -(29) BroadcastHashJoin [codegen id : 3] +(31) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_customer_sk#1] Right keys [1]: [c_customer_sk#27] Join type: Inner Join condition: None -(30) Project [codegen id : 3] +(32) 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] -(31) TakeOrderedAndProject +(33) 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 (36) -+- * ColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan parquet spark_catalog.default.date_dim (32) +BroadcastExchange (38) ++- * ColumnarToRow (37) + +- CometProject (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.date_dim (34) -(32) Scan parquet spark_catalog.default.date_dim +(34) 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 -(33) CometFilter +(35) 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)) -(34) CometProject +(36) CometProject Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(35) ColumnarToRow [codegen id : 1] +(37) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(36) BroadcastExchange +(38) BroadcastExchange Input [1]: [d_date_sk#10] 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 f2883e450..30a67a19b 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 @@ -3,40 +3,42 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, 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 (1) - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] - ColumnarToRow - InputAdapter - CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city,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_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_store_sk,s_city] - CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [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,d_date_sk] - CometFilter [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] - 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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [s_store_sk,s_city] #4 - CometProject [s_store_sk,s_city] - CometFilter [s_store_sk,s_number_employees,s_city] - CometScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] - CometBroadcastExchange [hd_demo_sk] #5 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + ColumnarToRow + InputAdapter + CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city] #1 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] + ColumnarToRow + InputAdapter + CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city,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_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_store_sk,s_city] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + CometBroadcastHashJoin [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,d_date_sk] + CometFilter [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] + 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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometBroadcastExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometBroadcastExchange [s_store_sk,s_city] #4 + CometProject [s_store_sk,s_city] + CometFilter [s_store_sk,s_number_employees,s_city] + CometScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] + CometBroadcastExchange [hd_demo_sk] #5 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #6 WholeStageCodegen (2) 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 07bfd877c..ac20b8b5b 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,48 +1,48 @@ == Physical Plan == 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) + +- * ColumnarToRow (42) + +- CometColumnarExchange (41) + +- RowToColumnar (40) + +- * HashAggregate (39) + +- * ColumnarToRow (38) + +- CometProject (37) + +- CometBroadcastHashJoin (36) + :- 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 (35) + +- CometHashAggregate (34) + +- CometColumnarExchange (33) + +- CometHashAggregate (32) + +- CometBroadcastHashJoin (31) + :- CometProject (16) + : +- CometFilter (15) + : +- CometScan parquet spark_catalog.default.customer_address (14) + +- CometBroadcastExchange (30) + +- CometProject (29) + +- CometFilter (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) + +- CometColumnarExchange (26) + +- CometHashAggregate (25) + +- CometProject (24) + +- CometBroadcastHashJoin (23) + :- CometFilter (18) + : +- CometScan parquet spark_catalog.default.customer_address (17) + +- CometBroadcastExchange (22) + +- CometProject (21) + +- CometFilter (20) + +- CometScan parquet spark_catalog.default.customer (19) (1) Scan parquet spark_catalog.default.store_sales @@ -109,155 +109,147 @@ Arguments: [ss_store_sk#1], [s_store_sk#8], Inner, BuildRight 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] -(14) ColumnarToRow [codegen id : 5] -Input [3]: [ss_net_profit#2, s_store_name#9, s_zip#10] - -(15) Scan parquet spark_catalog.default.customer_address +(14) 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 -(16) CometFilter +(15) 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))) -(17) CometProject +(16) CometProject Input [1]: [ca_zip#11] Arguments: [ca_zip#12], [substr(ca_zip#11, 1, 5) AS ca_zip#12] -(18) ColumnarToRow [codegen id : 3] -Input [1]: [ca_zip#12] - -(19) Scan parquet spark_catalog.default.customer_address +(17) 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 -(20) CometFilter +(18) CometFilter Input [2]: [ca_address_sk#13, ca_zip#14] Condition : isnotnull(ca_address_sk#13) -(21) Scan parquet spark_catalog.default.customer +(19) 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 -(22) CometFilter +(20) 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)) -(23) CometProject +(21) CometProject Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] Arguments: [c_current_addr_sk#15], [c_current_addr_sk#15] -(24) CometBroadcastExchange +(22) CometBroadcastExchange Input [1]: [c_current_addr_sk#15] Arguments: [c_current_addr_sk#15] -(25) CometBroadcastHashJoin +(23) 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 -(26) CometProject +(24) CometProject Input [3]: [ca_address_sk#13, ca_zip#14, c_current_addr_sk#15] Arguments: [ca_zip#14], [ca_zip#14] -(27) CometHashAggregate +(25) CometHashAggregate Input [1]: [ca_zip#14] Keys [1]: [ca_zip#14] Functions [1]: [partial_count(1)] -(28) ColumnarToRow [codegen id : 1] +(26) CometColumnarExchange Input [2]: [ca_zip#14, count#17] +Arguments: hashpartitioning(ca_zip#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(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] +(27) CometHashAggregate Input [2]: [ca_zip#14, count#17] Keys [1]: [ca_zip#14] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#18] -Results [2]: [substr(ca_zip#14, 1, 5) AS ca_zip#19, count(1)#18 AS cnt#20] -(31) Filter [codegen id : 2] -Input [2]: [ca_zip#19, cnt#20] -Condition : (cnt#20 > 10) +(28) CometFilter +Input [2]: [ca_zip#18, cnt#19] +Condition : (cnt#19 > 10) -(32) Project [codegen id : 2] -Output [1]: [ca_zip#19] -Input [2]: [ca_zip#19, cnt#20] +(29) CometProject +Input [2]: [ca_zip#18, cnt#19] +Arguments: [ca_zip#18], [ca_zip#18] -(33) BroadcastExchange -Input [1]: [ca_zip#19] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [plan_id=2] +(30) CometBroadcastExchange +Input [1]: [ca_zip#18] +Arguments: [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 +(31) CometBroadcastHashJoin +Left output [1]: [ca_zip#12] +Right output [1]: [ca_zip#18] +Arguments: [coalesce(ca_zip#12, ), isnull(ca_zip#12)], [coalesce(ca_zip#18, ), isnull(ca_zip#18)], LeftSemi, BuildRight -(35) HashAggregate [codegen id : 3] +(32) CometHashAggregate Input [1]: [ca_zip#12] Keys [1]: [ca_zip#12] Functions: [] -Aggregate Attributes: [] -Results [1]: [ca_zip#12] -(36) Exchange +(33) CometColumnarExchange Input [1]: [ca_zip#12] -Arguments: hashpartitioning(ca_zip#12, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(ca_zip#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(37) HashAggregate [codegen id : 4] +(34) CometHashAggregate Input [1]: [ca_zip#12] Keys [1]: [ca_zip#12] Functions: [] -Aggregate Attributes: [] -Results [1]: [ca_zip#12] -(38) BroadcastExchange +(35) CometBroadcastExchange Input [1]: [ca_zip#12] -Arguments: HashedRelationBroadcastMode(List(substr(input[0, string, true], 1, 2)),false), [plan_id=4] +Arguments: [ca_zip#12] -(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 +(36) CometBroadcastHashJoin +Left output [3]: [ss_net_profit#2, s_store_name#9, s_zip#10] +Right output [1]: [ca_zip#12] +Arguments: [substr(s_zip#10, 1, 2)], [substr(ca_zip#12, 1, 2)], Inner, BuildRight -(40) Project [codegen id : 5] -Output [2]: [ss_net_profit#2, s_store_name#9] +(37) CometProject Input [4]: [ss_net_profit#2, s_store_name#9, s_zip#10, ca_zip#12] +Arguments: [ss_net_profit#2, s_store_name#9], [ss_net_profit#2, s_store_name#9] + +(38) ColumnarToRow [codegen id : 1] +Input [2]: [ss_net_profit#2, s_store_name#9] -(41) HashAggregate [codegen id : 5] +(39) HashAggregate [codegen id : 1] 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#21] -Results [2]: [s_store_name#9, sum#22] +Aggregate Attributes [1]: [sum#20] +Results [2]: [s_store_name#9, sum#21] + +(40) RowToColumnar +Input [2]: [s_store_name#9, sum#21] + +(41) CometColumnarExchange +Input [2]: [s_store_name#9, sum#21] +Arguments: hashpartitioning(s_store_name#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(42) Exchange -Input [2]: [s_store_name#9, sum#22] -Arguments: hashpartitioning(s_store_name#9, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(42) ColumnarToRow [codegen id : 2] +Input [2]: [s_store_name#9, sum#21] -(43) HashAggregate [codegen id : 6] -Input [2]: [s_store_name#9, sum#22] +(43) HashAggregate [codegen id : 2] +Input [2]: [s_store_name#9, sum#21] Keys [1]: [s_store_name#9] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -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] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#22] +Results [2]: [s_store_name#9, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#22,17,2) AS 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] +Input [2]: [s_store_name#9, sum(ss_net_profit)#23] +Arguments: 100, [s_store_name#9 ASC NULLS FIRST], [s_store_name#9, sum(ss_net_profit)#23] ===== Subqueries ===== @@ -289,6 +281,6 @@ Input [1]: [d_date_sk#5] (49) BroadcastExchange Input [1]: [d_date_sk#5] -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/q8/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt index 637f3b7ba..2b89ebb92 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,66 +1,56 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [s_store_name,sum] [sum(UnscaledValue(ss_net_profit)),sum(ss_net_profit),sum] - InputAdapter - Exchange [s_store_name] #1 - WholeStageCodegen (5) - HashAggregate [s_store_name,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_store_name] - BroadcastHashJoin [s_zip,ca_zip] + ColumnarToRow + InputAdapter + CometColumnarExchange [s_store_name] #1 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [s_store_name,ss_net_profit] [sum,sum] ColumnarToRow InputAdapter - CometProject [ss_net_profit,s_store_name,s_zip] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,s_store_sk,s_store_name,s_zip] - CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #4 - CometFilter [s_store_sk,s_store_name,s_zip] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - HashAggregate [ca_zip] - InputAdapter - Exchange [ca_zip] #6 - WholeStageCodegen (3) - HashAggregate [ca_zip] - BroadcastHashJoin [ca_zip,ca_zip] - ColumnarToRow - InputAdapter - CometProject [ca_zip] [ca_zip] - CometFilter [ca_zip] - CometScan parquet spark_catalog.default.customer_address [ca_zip] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - Project [ca_zip] - Filter [cnt] - HashAggregate [ca_zip,count] [count(1),ca_zip,cnt,count] - InputAdapter - Exchange [ca_zip] #8 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ca_zip,count] - CometProject [ca_zip] - CometBroadcastHashJoin [ca_address_sk,ca_zip,c_current_addr_sk] - CometFilter [ca_address_sk,ca_zip] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange [c_current_addr_sk] #9 - CometProject [c_current_addr_sk] - CometFilter [c_current_addr_sk,c_preferred_cust_flag] - CometScan parquet spark_catalog.default.customer [c_current_addr_sk,c_preferred_cust_flag] + CometProject [ss_net_profit,s_store_name] + CometBroadcastHashJoin [ss_net_profit,s_store_name,s_zip,ca_zip] + CometProject [ss_net_profit,s_store_name,s_zip] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,s_store_sk,s_store_name,s_zip] + CometProject [ss_store_sk,ss_net_profit] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #4 + CometFilter [s_store_sk,s_store_name,s_zip] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] + CometBroadcastExchange [ca_zip] #5 + CometHashAggregate [ca_zip] + CometColumnarExchange [ca_zip] #6 + CometHashAggregate [ca_zip] + CometBroadcastHashJoin [ca_zip,ca_zip] + CometProject [ca_zip] [ca_zip] + CometFilter [ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_zip] + CometBroadcastExchange [ca_zip] #7 + CometProject [ca_zip] + CometFilter [ca_zip,cnt] + CometHashAggregate [ca_zip,cnt,ca_zip,count,count(1)] + CometColumnarExchange [ca_zip] #8 + CometHashAggregate [ca_zip,count] + CometProject [ca_zip] + CometBroadcastHashJoin [ca_address_sk,ca_zip,c_current_addr_sk] + CometFilter [ca_address_sk,ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] + CometBroadcastExchange [c_current_addr_sk] #9 + CometProject [c_current_addr_sk] + CometFilter [c_current_addr_sk,c_preferred_cust_flag] + 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/q80/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/explain.txt index db2d015db..97db908a7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/explain.txt @@ -1,111 +1,114 @@ == Physical Plan == -TakeOrderedAndProject (107) -+- * HashAggregate (106) - +- Exchange (105) - +- * HashAggregate (104) - +- * Expand (103) - +- Union (102) - :- * HashAggregate (39) - : +- Exchange (38) - : +- * HashAggregate (37) - : +- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * Project (29) - : : +- * BroadcastHashJoin Inner BuildRight (28) - : : :- * Project (22) - : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : :- * Project (16) - : : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : : :- * Project (13) - : : : : : +- * SortMergeJoin LeftOuter (12) - : : : : : :- * Sort (5) - : : : : : : +- Exchange (4) - : : : : : : +- * ColumnarToRow (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : +- * Sort (11) - : : : : : +- Exchange (10) - : : : : : +- * ColumnarToRow (9) - : : : : : +- CometProject (8) - : : : : : +- CometFilter (7) - : : : : : +- CometScan parquet spark_catalog.default.store_returns (6) - : : : : +- ReusedExchange (14) - : : : +- BroadcastExchange (20) - : : : +- * ColumnarToRow (19) - : : : +- CometFilter (18) - : : : +- CometScan parquet spark_catalog.default.store (17) - : : +- BroadcastExchange (27) - : : +- * ColumnarToRow (26) - : : +- CometProject (25) - : : +- CometFilter (24) - : : +- CometScan parquet spark_catalog.default.item (23) - : +- BroadcastExchange (34) - : +- * ColumnarToRow (33) - : +- CometProject (32) - : +- CometFilter (31) - : +- CometScan parquet spark_catalog.default.promotion (30) - :- * HashAggregate (70) - : +- Exchange (69) - : +- * HashAggregate (68) - : +- * Project (67) - : +- * BroadcastHashJoin Inner BuildRight (66) - : :- * Project (64) - : : +- * BroadcastHashJoin Inner BuildRight (63) - : : :- * Project (61) - : : : +- * BroadcastHashJoin Inner BuildRight (60) - : : : :- * Project (55) - : : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : : :- * Project (52) - : : : : : +- * SortMergeJoin LeftOuter (51) - : : : : : :- * Sort (44) - : : : : : : +- Exchange (43) - : : : : : : +- * ColumnarToRow (42) - : : : : : : +- CometFilter (41) - : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (40) - : : : : : +- * Sort (50) - : : : : : +- Exchange (49) - : : : : : +- * ColumnarToRow (48) - : : : : : +- CometProject (47) - : : : : : +- CometFilter (46) - : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (45) - : : : : +- ReusedExchange (53) - : : : +- BroadcastExchange (59) - : : : +- * ColumnarToRow (58) - : : : +- CometFilter (57) - : : : +- CometScan parquet spark_catalog.default.catalog_page (56) - : : +- ReusedExchange (62) - : +- ReusedExchange (65) - +- * HashAggregate (101) - +- Exchange (100) - +- * HashAggregate (99) - +- * Project (98) - +- * BroadcastHashJoin Inner BuildRight (97) - :- * Project (95) - : +- * BroadcastHashJoin Inner BuildRight (94) - : :- * Project (92) - : : +- * BroadcastHashJoin Inner BuildRight (91) - : : :- * Project (86) - : : : +- * BroadcastHashJoin Inner BuildRight (85) - : : : :- * Project (83) - : : : : +- * SortMergeJoin LeftOuter (82) - : : : : :- * Sort (75) - : : : : : +- Exchange (74) - : : : : : +- * ColumnarToRow (73) - : : : : : +- CometFilter (72) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (71) - : : : : +- * Sort (81) - : : : : +- Exchange (80) - : : : : +- * ColumnarToRow (79) - : : : : +- CometProject (78) - : : : : +- CometFilter (77) - : : : : +- CometScan parquet spark_catalog.default.web_returns (76) - : : : +- ReusedExchange (84) - : : +- BroadcastExchange (90) - : : +- * ColumnarToRow (89) - : : +- CometFilter (88) - : : +- CometScan parquet spark_catalog.default.web_site (87) - : +- ReusedExchange (93) - +- ReusedExchange (96) +TakeOrderedAndProject (110) ++- * HashAggregate (109) + +- * ColumnarToRow (108) + +- CometColumnarExchange (107) + +- RowToColumnar (106) + +- * HashAggregate (105) + +- * Expand (104) + +- Union (103) + :- * HashAggregate (40) + : +- * ColumnarToRow (39) + : +- CometColumnarExchange (38) + : +- RowToColumnar (37) + : +- * HashAggregate (36) + : +- * ColumnarToRow (35) + : +- CometProject (34) + : +- CometBroadcastHashJoin (33) + : :- CometProject (28) + : : +- CometBroadcastHashJoin (27) + : : :- CometProject (22) + : : : +- CometBroadcastHashJoin (21) + : : : :- CometProject (17) + : : : : +- CometBroadcastHashJoin (16) + : : : : :- CometProject (11) + : : : : : +- CometSortMergeJoin (10) + : : : : : :- CometSort (4) + : : : : : : +- CometColumnarExchange (3) + : : : : : : +- CometFilter (2) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : +- CometSort (9) + : : : : : +- CometColumnarExchange (8) + : : : : : +- CometProject (7) + : : : : : +- CometFilter (6) + : : : : : +- CometScan parquet spark_catalog.default.store_returns (5) + : : : : +- CometBroadcastExchange (15) + : : : : +- CometProject (14) + : : : : +- CometFilter (13) + : : : : +- CometScan parquet spark_catalog.default.date_dim (12) + : : : +- CometBroadcastExchange (20) + : : : +- CometFilter (19) + : : : +- CometScan parquet spark_catalog.default.store (18) + : : +- CometBroadcastExchange (26) + : : +- CometProject (25) + : : +- CometFilter (24) + : : +- CometScan parquet spark_catalog.default.item (23) + : +- CometBroadcastExchange (32) + : +- CometProject (31) + : +- CometFilter (30) + : +- CometScan parquet spark_catalog.default.promotion (29) + :- * HashAggregate (71) + : +- * ColumnarToRow (70) + : +- CometColumnarExchange (69) + : +- RowToColumnar (68) + : +- * HashAggregate (67) + : +- * ColumnarToRow (66) + : +- CometProject (65) + : +- CometBroadcastHashJoin (64) + : :- CometProject (62) + : : +- CometBroadcastHashJoin (61) + : : :- CometProject (59) + : : : +- CometBroadcastHashJoin (58) + : : : :- CometProject (54) + : : : : +- CometBroadcastHashJoin (53) + : : : : :- CometProject (51) + : : : : : +- CometSortMergeJoin (50) + : : : : : :- CometSort (44) + : : : : : : +- CometColumnarExchange (43) + : : : : : : +- CometFilter (42) + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (41) + : : : : : +- CometSort (49) + : : : : : +- CometColumnarExchange (48) + : : : : : +- CometProject (47) + : : : : : +- CometFilter (46) + : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (45) + : : : : +- ReusedExchange (52) + : : : +- CometBroadcastExchange (57) + : : : +- CometFilter (56) + : : : +- CometScan parquet spark_catalog.default.catalog_page (55) + : : +- ReusedExchange (60) + : +- ReusedExchange (63) + +- * HashAggregate (102) + +- * ColumnarToRow (101) + +- CometColumnarExchange (100) + +- RowToColumnar (99) + +- * HashAggregate (98) + +- * ColumnarToRow (97) + +- CometProject (96) + +- CometBroadcastHashJoin (95) + :- CometProject (93) + : +- CometBroadcastHashJoin (92) + : :- CometProject (90) + : : +- CometBroadcastHashJoin (89) + : : :- CometProject (85) + : : : +- CometBroadcastHashJoin (84) + : : : :- CometProject (82) + : : : : +- CometSortMergeJoin (81) + : : : : :- CometSort (75) + : : : : : +- CometColumnarExchange (74) + : : : : : +- CometFilter (73) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (72) + : : : : +- CometSort (80) + : : : : +- CometColumnarExchange (79) + : : : : +- CometProject (78) + : : : : +- CometFilter (77) + : : : : +- CometScan parquet spark_catalog.default.web_returns (76) + : : : +- ReusedExchange (83) + : : +- CometBroadcastExchange (88) + : : +- CometFilter (87) + : : +- CometScan parquet spark_catalog.default.web_site (86) + : +- ReusedExchange (91) + +- ReusedExchange (94) (1) Scan parquet spark_catalog.default.store_sales @@ -120,526 +123,527 @@ ReadSchema: struct -(7) CometFilter +(6) CometFilter Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] Condition : (isnotnull(sr_item_sk#9) AND isnotnull(sr_ticket_number#10)) -(8) CometProject +(7) CometProject Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -(9) ColumnarToRow [codegen id : 3] -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] - -(10) Exchange +(8) CometColumnarExchange Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(11) Sort [codegen id : 4] +(9) CometSort Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST], false, 0 +Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST] -(12) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_item_sk#1, ss_ticket_number#4] -Right keys [2]: [sr_item_sk#9, sr_ticket_number#10] -Join type: LeftOuter -Join condition: None +(10) CometSortMergeJoin +Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] +Right output [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] +Arguments: [ss_item_sk#1, ss_ticket_number#4], [sr_item_sk#9, sr_ticket_number#10], LeftOuter -(13) Project [codegen id : 9] -Output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] +(11) CometProject Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] +Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] -(14) ReusedExchange [Reuses operator id: 112] -Output [1]: [d_date_sk#14] +(12) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#15] +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-22), IsNotNull(d_date_sk)] +ReadSchema: struct -(15) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None +(13) CometFilter +Input [2]: [d_date_sk#14, d_date#15] +Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#15 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) -(16) Project [codegen id : 9] -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] +(14) CometProject +Input [2]: [d_date_sk#14, d_date#15] +Arguments: [d_date_sk#14], [d_date_sk#14] + +(15) CometBroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: [d_date_sk#14] + +(16) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] +Right output [1]: [d_date_sk#14] +Arguments: [ss_sold_date_sk#7], [d_date_sk#14], Inner, BuildRight + +(17) CometProject Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#14] +Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] -(17) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#15, s_store_id#16] +(18) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#16, s_store_id#17] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(18) CometFilter -Input [2]: [s_store_sk#15, s_store_id#16] -Condition : isnotnull(s_store_sk#15) - -(19) ColumnarToRow [codegen id : 6] -Input [2]: [s_store_sk#15, s_store_id#16] +(19) CometFilter +Input [2]: [s_store_sk#16, s_store_id#17] +Condition : isnotnull(s_store_sk#16) -(20) BroadcastExchange -Input [2]: [s_store_sk#15, s_store_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(20) CometBroadcastExchange +Input [2]: [s_store_sk#16, s_store_id#17] +Arguments: [s_store_sk#16, s_store_id#17] -(21) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#15] -Join type: Inner -Join condition: None +(21) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] +Right output [2]: [s_store_sk#16, s_store_id#17] +Arguments: [ss_store_sk#2], [s_store_sk#16], Inner, BuildRight -(22) Project [codegen id : 9] -Output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#15, s_store_id#16] +(22) CometProject +Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#16, s_store_id#17] +Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17], [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] (23) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#17, i_current_price#18] +Output [2]: [i_item_sk#18, i_current_price#19] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), GreaterThan(i_current_price,50.00), IsNotNull(i_item_sk)] ReadSchema: struct (24) CometFilter -Input [2]: [i_item_sk#17, i_current_price#18] -Condition : ((isnotnull(i_current_price#18) AND (i_current_price#18 > 50.00)) AND isnotnull(i_item_sk#17)) +Input [2]: [i_item_sk#18, i_current_price#19] +Condition : ((isnotnull(i_current_price#19) AND (i_current_price#19 > 50.00)) AND isnotnull(i_item_sk#18)) (25) CometProject -Input [2]: [i_item_sk#17, i_current_price#18] -Arguments: [i_item_sk#17], [i_item_sk#17] - -(26) ColumnarToRow [codegen id : 7] -Input [1]: [i_item_sk#17] +Input [2]: [i_item_sk#18, i_current_price#19] +Arguments: [i_item_sk#18], [i_item_sk#18] -(27) BroadcastExchange -Input [1]: [i_item_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(26) CometBroadcastExchange +Input [1]: [i_item_sk#18] +Arguments: [i_item_sk#18] -(28) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#17] -Join type: Inner -Join condition: None +(27) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] +Right output [1]: [i_item_sk#18] +Arguments: [ss_item_sk#1], [i_item_sk#18], Inner, BuildRight -(29) Project [codegen id : 9] -Output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16, i_item_sk#17] +(28) CometProject +Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17, i_item_sk#18] +Arguments: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17], [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] -(30) Scan parquet spark_catalog.default.promotion -Output [2]: [p_promo_sk#19, p_channel_tv#20] +(29) Scan parquet spark_catalog.default.promotion +Output [2]: [p_promo_sk#20, p_channel_tv#21] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_channel_tv), EqualTo(p_channel_tv,N), IsNotNull(p_promo_sk)] ReadSchema: struct -(31) CometFilter -Input [2]: [p_promo_sk#19, p_channel_tv#20] -Condition : ((isnotnull(p_channel_tv#20) AND (p_channel_tv#20 = N)) AND isnotnull(p_promo_sk#19)) +(30) CometFilter +Input [2]: [p_promo_sk#20, p_channel_tv#21] +Condition : ((isnotnull(p_channel_tv#21) AND (p_channel_tv#21 = N)) AND isnotnull(p_promo_sk#20)) -(32) CometProject -Input [2]: [p_promo_sk#19, p_channel_tv#20] -Arguments: [p_promo_sk#19], [p_promo_sk#19] +(31) CometProject +Input [2]: [p_promo_sk#20, p_channel_tv#21] +Arguments: [p_promo_sk#20], [p_promo_sk#20] -(33) ColumnarToRow [codegen id : 8] -Input [1]: [p_promo_sk#19] +(32) CometBroadcastExchange +Input [1]: [p_promo_sk#20] +Arguments: [p_promo_sk#20] -(34) BroadcastExchange -Input [1]: [p_promo_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(33) CometBroadcastHashJoin +Left output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] +Right output [1]: [p_promo_sk#20] +Arguments: [ss_promo_sk#3], [p_promo_sk#20], Inner, BuildRight -(35) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_promo_sk#3] -Right keys [1]: [p_promo_sk#19] -Join type: Inner -Join condition: None +(34) CometProject +Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17, p_promo_sk#20] +Arguments: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17], [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] -(36) Project [codegen id : 9] -Output [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] -Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16, p_promo_sk#19] +(35) ColumnarToRow [codegen id : 1] +Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] -(37) HashAggregate [codegen id : 9] -Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] -Keys [1]: [s_store_id#16] +(36) HashAggregate [codegen id : 1] +Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] +Keys [1]: [s_store_id#17] Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] -Aggregate Attributes [5]: [sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -Results [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] +Aggregate Attributes [5]: [sum#22, sum#23, isEmpty#24, sum#25, isEmpty#26] +Results [6]: [s_store_id#17, sum#27, sum#28, isEmpty#29, sum#30, isEmpty#31] -(38) Exchange -Input [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] -Arguments: hashpartitioning(s_store_id#16, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(37) RowToColumnar +Input [6]: [s_store_id#17, sum#27, sum#28, isEmpty#29, sum#30, isEmpty#31] -(39) HashAggregate [codegen id : 10] -Input [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] -Keys [1]: [s_store_id#16] +(38) CometColumnarExchange +Input [6]: [s_store_id#17, sum#27, sum#28, isEmpty#29, sum#30, isEmpty#31] +Arguments: hashpartitioning(s_store_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(39) ColumnarToRow [codegen id : 2] +Input [6]: [s_store_id#17, sum#27, sum#28, isEmpty#29, sum#30, isEmpty#31] + +(40) HashAggregate [codegen id : 2] +Input [6]: [s_store_id#17, sum#27, sum#28, isEmpty#29, sum#30, isEmpty#31] +Keys [1]: [s_store_id#17] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#5))#31, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#32, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#33] -Results [5]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#31,17,2) AS sales#34, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#32 AS returns#35, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#33 AS profit#36, store channel AS channel#37, concat(store, s_store_id#16) AS id#38] +Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#5))#32, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#33, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#34] +Results [5]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#32,17,2) AS sales#35, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#33 AS returns#36, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#34 AS profit#37, store channel AS channel#38, concat(store, s_store_id#17) AS id#39] -(40) Scan parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +(41) Scan parquet spark_catalog.default.catalog_sales +Output [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#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), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] ReadSchema: struct -(41) CometFilter -Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Condition : ((isnotnull(cs_catalog_page_sk#39) AND isnotnull(cs_item_sk#40)) AND isnotnull(cs_promo_sk#41)) +(42) CometFilter +Input [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] +Condition : ((isnotnull(cs_catalog_page_sk#40) AND isnotnull(cs_item_sk#41)) AND isnotnull(cs_promo_sk#42)) -(42) ColumnarToRow [codegen id : 11] -Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +(43) CometColumnarExchange +Input [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] +Arguments: hashpartitioning(cs_item_sk#41, cs_order_number#43, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(43) Exchange -Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Arguments: hashpartitioning(cs_item_sk#40, cs_order_number#42, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(44) Sort [codegen id : 12] -Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Arguments: [cs_item_sk#40 ASC NULLS FIRST, cs_order_number#42 ASC NULLS FIRST], false, 0 +(44) CometSort +Input [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] +Arguments: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46], [cs_item_sk#41 ASC NULLS FIRST, cs_order_number#43 ASC NULLS FIRST] (45) Scan parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] +Output [5]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51, cr_returned_date_sk#52] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct (46) CometFilter -Input [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] -Condition : (isnotnull(cr_item_sk#47) AND isnotnull(cr_order_number#48)) +Input [5]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51, cr_returned_date_sk#52] +Condition : (isnotnull(cr_item_sk#48) AND isnotnull(cr_order_number#49)) (47) CometProject -Input [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] -Arguments: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50], [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] - -(48) ColumnarToRow [codegen id : 13] -Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] +Input [5]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51, cr_returned_date_sk#52] +Arguments: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51], [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51] -(49) Exchange -Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -Arguments: hashpartitioning(cr_item_sk#47, cr_order_number#48, 5), ENSURE_REQUIREMENTS, [plan_id=8] +(48) CometColumnarExchange +Input [4]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51] +Arguments: hashpartitioning(cr_item_sk#48, cr_order_number#49, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(50) Sort [codegen id : 14] -Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -Arguments: [cr_item_sk#47 ASC NULLS FIRST, cr_order_number#48 ASC NULLS FIRST], false, 0 +(49) CometSort +Input [4]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51] +Arguments: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51], [cr_item_sk#48 ASC NULLS FIRST, cr_order_number#49 ASC NULLS FIRST] -(51) SortMergeJoin [codegen id : 19] -Left keys [2]: [cs_item_sk#40, cs_order_number#42] -Right keys [2]: [cr_item_sk#47, cr_order_number#48] -Join type: LeftOuter -Join condition: None +(50) CometSortMergeJoin +Left output [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] +Right output [4]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51] +Arguments: [cs_item_sk#41, cs_order_number#43], [cr_item_sk#48, cr_order_number#49], LeftOuter -(52) Project [codegen id : 19] -Output [8]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50] -Input [11]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] +(51) CometProject +Input [11]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46, cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51] +Arguments: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46, cr_return_amount#50, cr_net_loss#51], [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46, cr_return_amount#50, cr_net_loss#51] -(53) ReusedExchange [Reuses operator id: 112] -Output [1]: [d_date_sk#52] +(52) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#53] -(54) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_sold_date_sk#45] -Right keys [1]: [d_date_sk#52] -Join type: Inner -Join condition: None +(53) CometBroadcastHashJoin +Left output [8]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46, cr_return_amount#50, cr_net_loss#51] +Right output [1]: [d_date_sk#53] +Arguments: [cs_sold_date_sk#46], [d_date_sk#53], Inner, BuildRight -(55) Project [codegen id : 19] -Output [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50] -Input [9]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50, d_date_sk#52] +(54) CometProject +Input [9]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46, cr_return_amount#50, cr_net_loss#51, d_date_sk#53] +Arguments: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51], [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51] -(56) Scan parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +(55) Scan parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#54, cp_catalog_page_id#55] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(57) CometFilter -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Condition : isnotnull(cp_catalog_page_sk#53) - -(58) ColumnarToRow [codegen id : 16] -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] - -(59) BroadcastExchange -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -(60) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_catalog_page_sk#39] -Right keys [1]: [cp_catalog_page_sk#53] -Join type: Inner -Join condition: None - -(61) Project [codegen id : 19] -Output [7]: [cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] -Input [9]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_sk#53, cp_catalog_page_id#54] - -(62) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#55] - -(63) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_item_sk#40] -Right keys [1]: [i_item_sk#55] -Join type: Inner -Join condition: None - -(64) Project [codegen id : 19] -Output [6]: [cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] -Input [8]: [cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54, i_item_sk#55] - -(65) ReusedExchange [Reuses operator id: 34] -Output [1]: [p_promo_sk#56] - -(66) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_promo_sk#41] -Right keys [1]: [p_promo_sk#56] -Join type: Inner -Join condition: None - -(67) Project [codegen id : 19] -Output [5]: [cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] -Input [7]: [cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54, p_promo_sk#56] - -(68) HashAggregate [codegen id : 19] -Input [5]: [cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] -Keys [1]: [cp_catalog_page_id#54] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#43)), partial_sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))] -Aggregate Attributes [5]: [sum#57, sum#58, isEmpty#59, sum#60, isEmpty#61] -Results [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] - -(69) Exchange -Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] -Arguments: hashpartitioning(cp_catalog_page_id#54, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(70) HashAggregate [codegen id : 20] -Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] -Keys [1]: [cp_catalog_page_id#54] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#43)), sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00)), sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#43))#67, sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00))#68, sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))#69] -Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#43))#67,17,2) AS sales#70, sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00))#68 AS returns#71, sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))#69 AS profit#72, catalog channel AS channel#73, concat(catalog_page, cp_catalog_page_id#54) AS id#74] - -(71) Scan parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] +(56) CometFilter +Input [2]: [cp_catalog_page_sk#54, cp_catalog_page_id#55] +Condition : isnotnull(cp_catalog_page_sk#54) + +(57) CometBroadcastExchange +Input [2]: [cp_catalog_page_sk#54, cp_catalog_page_id#55] +Arguments: [cp_catalog_page_sk#54, cp_catalog_page_id#55] + +(58) CometBroadcastHashJoin +Left output [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51] +Right output [2]: [cp_catalog_page_sk#54, cp_catalog_page_id#55] +Arguments: [cs_catalog_page_sk#40], [cp_catalog_page_sk#54], Inner, BuildRight + +(59) CometProject +Input [9]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_sk#54, cp_catalog_page_id#55] +Arguments: [cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#55], [cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#55] + +(60) ReusedExchange [Reuses operator id: 26] +Output [1]: [i_item_sk#56] + +(61) CometBroadcastHashJoin +Left output [7]: [cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#55] +Right output [1]: [i_item_sk#56] +Arguments: [cs_item_sk#41], [i_item_sk#56], Inner, BuildRight + +(62) CometProject +Input [8]: [cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#55, i_item_sk#56] +Arguments: [cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#55], [cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#55] + +(63) ReusedExchange [Reuses operator id: 32] +Output [1]: [p_promo_sk#57] + +(64) CometBroadcastHashJoin +Left output [6]: [cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#55] +Right output [1]: [p_promo_sk#57] +Arguments: [cs_promo_sk#42], [p_promo_sk#57], Inner, BuildRight + +(65) CometProject +Input [7]: [cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#55, p_promo_sk#57] +Arguments: [cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#55], [cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#55] + +(66) ColumnarToRow [codegen id : 3] +Input [5]: [cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#55] + +(67) HashAggregate [codegen id : 3] +Input [5]: [cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#55] +Keys [1]: [cp_catalog_page_id#55] +Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#44)), partial_sum(coalesce(cast(cr_return_amount#50 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#45 - coalesce(cast(cr_net_loss#51 as decimal(12,2)), 0.00)))] +Aggregate Attributes [5]: [sum#58, sum#59, isEmpty#60, sum#61, isEmpty#62] +Results [6]: [cp_catalog_page_id#55, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] + +(68) RowToColumnar +Input [6]: [cp_catalog_page_id#55, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] + +(69) CometColumnarExchange +Input [6]: [cp_catalog_page_id#55, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] +Arguments: hashpartitioning(cp_catalog_page_id#55, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(70) ColumnarToRow [codegen id : 4] +Input [6]: [cp_catalog_page_id#55, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] + +(71) HashAggregate [codegen id : 4] +Input [6]: [cp_catalog_page_id#55, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] +Keys [1]: [cp_catalog_page_id#55] +Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#44)), sum(coalesce(cast(cr_return_amount#50 as decimal(12,2)), 0.00)), sum((cs_net_profit#45 - coalesce(cast(cr_net_loss#51 as decimal(12,2)), 0.00)))] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#44))#68, sum(coalesce(cast(cr_return_amount#50 as decimal(12,2)), 0.00))#69, sum((cs_net_profit#45 - coalesce(cast(cr_net_loss#51 as decimal(12,2)), 0.00)))#70] +Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#44))#68,17,2) AS sales#71, sum(coalesce(cast(cr_return_amount#50 as decimal(12,2)), 0.00))#69 AS returns#72, sum((cs_net_profit#45 - coalesce(cast(cr_net_loss#51 as decimal(12,2)), 0.00)))#70 AS profit#73, catalog channel AS channel#74, concat(catalog_page, cp_catalog_page_id#55) AS id#75] + +(72) Scan parquet spark_catalog.default.web_sales +Output [7]: [ws_item_sk#76, ws_web_site_sk#77, ws_promo_sk#78, ws_order_number#79, ws_ext_sales_price#80, ws_net_profit#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_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct -(72) CometFilter -Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Condition : ((isnotnull(ws_web_site_sk#76) AND isnotnull(ws_item_sk#75)) AND isnotnull(ws_promo_sk#77)) - -(73) ColumnarToRow [codegen id : 21] -Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] +(73) CometFilter +Input [7]: [ws_item_sk#76, ws_web_site_sk#77, ws_promo_sk#78, ws_order_number#79, ws_ext_sales_price#80, ws_net_profit#81, ws_sold_date_sk#82] +Condition : ((isnotnull(ws_web_site_sk#77) AND isnotnull(ws_item_sk#76)) AND isnotnull(ws_promo_sk#78)) -(74) Exchange -Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Arguments: hashpartitioning(ws_item_sk#75, ws_order_number#78, 5), ENSURE_REQUIREMENTS, [plan_id=11] +(74) CometColumnarExchange +Input [7]: [ws_item_sk#76, ws_web_site_sk#77, ws_promo_sk#78, ws_order_number#79, ws_ext_sales_price#80, ws_net_profit#81, ws_sold_date_sk#82] +Arguments: hashpartitioning(ws_item_sk#76, ws_order_number#79, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(75) Sort [codegen id : 22] -Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Arguments: [ws_item_sk#75 ASC NULLS FIRST, ws_order_number#78 ASC NULLS FIRST], false, 0 +(75) CometSort +Input [7]: [ws_item_sk#76, ws_web_site_sk#77, ws_promo_sk#78, ws_order_number#79, ws_ext_sales_price#80, ws_net_profit#81, ws_sold_date_sk#82] +Arguments: [ws_item_sk#76, ws_web_site_sk#77, ws_promo_sk#78, ws_order_number#79, ws_ext_sales_price#80, ws_net_profit#81, ws_sold_date_sk#82], [ws_item_sk#76 ASC NULLS FIRST, ws_order_number#79 ASC NULLS FIRST] (76) Scan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Output [5]: [wr_item_sk#84, wr_order_number#85, wr_return_amt#86, wr_net_loss#87, wr_returned_date_sk#88] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (77) CometFilter -Input [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] -Condition : (isnotnull(wr_item_sk#83) AND isnotnull(wr_order_number#84)) +Input [5]: [wr_item_sk#84, wr_order_number#85, wr_return_amt#86, wr_net_loss#87, wr_returned_date_sk#88] +Condition : (isnotnull(wr_item_sk#84) AND isnotnull(wr_order_number#85)) (78) CometProject -Input [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] -Arguments: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86], [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] +Input [5]: [wr_item_sk#84, wr_order_number#85, wr_return_amt#86, wr_net_loss#87, wr_returned_date_sk#88] +Arguments: [wr_item_sk#84, wr_order_number#85, wr_return_amt#86, wr_net_loss#87], [wr_item_sk#84, wr_order_number#85, wr_return_amt#86, wr_net_loss#87] -(79) ColumnarToRow [codegen id : 23] -Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] +(79) CometColumnarExchange +Input [4]: [wr_item_sk#84, wr_order_number#85, wr_return_amt#86, wr_net_loss#87] +Arguments: hashpartitioning(wr_item_sk#84, wr_order_number#85, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(80) Exchange -Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -Arguments: hashpartitioning(wr_item_sk#83, wr_order_number#84, 5), ENSURE_REQUIREMENTS, [plan_id=12] +(80) CometSort +Input [4]: [wr_item_sk#84, wr_order_number#85, wr_return_amt#86, wr_net_loss#87] +Arguments: [wr_item_sk#84, wr_order_number#85, wr_return_amt#86, wr_net_loss#87], [wr_item_sk#84 ASC NULLS FIRST, wr_order_number#85 ASC NULLS FIRST] -(81) Sort [codegen id : 24] -Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -Arguments: [wr_item_sk#83 ASC NULLS FIRST, wr_order_number#84 ASC NULLS FIRST], false, 0 +(81) CometSortMergeJoin +Left output [7]: [ws_item_sk#76, ws_web_site_sk#77, ws_promo_sk#78, ws_order_number#79, ws_ext_sales_price#80, ws_net_profit#81, ws_sold_date_sk#82] +Right output [4]: [wr_item_sk#84, wr_order_number#85, wr_return_amt#86, wr_net_loss#87] +Arguments: [ws_item_sk#76, ws_order_number#79], [wr_item_sk#84, wr_order_number#85], LeftOuter -(82) SortMergeJoin [codegen id : 29] -Left keys [2]: [ws_item_sk#75, ws_order_number#78] -Right keys [2]: [wr_item_sk#83, wr_order_number#84] -Join type: LeftOuter -Join condition: None +(82) CometProject +Input [11]: [ws_item_sk#76, ws_web_site_sk#77, ws_promo_sk#78, ws_order_number#79, ws_ext_sales_price#80, ws_net_profit#81, ws_sold_date_sk#82, wr_item_sk#84, wr_order_number#85, wr_return_amt#86, wr_net_loss#87] +Arguments: [ws_item_sk#76, ws_web_site_sk#77, ws_promo_sk#78, ws_ext_sales_price#80, ws_net_profit#81, ws_sold_date_sk#82, wr_return_amt#86, wr_net_loss#87], [ws_item_sk#76, ws_web_site_sk#77, ws_promo_sk#78, ws_ext_sales_price#80, ws_net_profit#81, ws_sold_date_sk#82, wr_return_amt#86, wr_net_loss#87] -(83) Project [codegen id : 29] -Output [8]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86] -Input [11]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] +(83) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#89] -(84) ReusedExchange [Reuses operator id: 112] -Output [1]: [d_date_sk#88] +(84) CometBroadcastHashJoin +Left output [8]: [ws_item_sk#76, ws_web_site_sk#77, ws_promo_sk#78, ws_ext_sales_price#80, ws_net_profit#81, ws_sold_date_sk#82, wr_return_amt#86, wr_net_loss#87] +Right output [1]: [d_date_sk#89] +Arguments: [ws_sold_date_sk#82], [d_date_sk#89], Inner, BuildRight -(85) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_sold_date_sk#81] -Right keys [1]: [d_date_sk#88] -Join type: Inner -Join condition: None +(85) CometProject +Input [9]: [ws_item_sk#76, ws_web_site_sk#77, ws_promo_sk#78, ws_ext_sales_price#80, ws_net_profit#81, ws_sold_date_sk#82, wr_return_amt#86, wr_net_loss#87, d_date_sk#89] +Arguments: [ws_item_sk#76, ws_web_site_sk#77, ws_promo_sk#78, ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87], [ws_item_sk#76, ws_web_site_sk#77, ws_promo_sk#78, ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87] -(86) Project [codegen id : 29] -Output [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86] -Input [9]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86, d_date_sk#88] - -(87) Scan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#89, web_site_id#90] +(86) Scan parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#90, web_site_id#91] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(88) CometFilter -Input [2]: [web_site_sk#89, web_site_id#90] -Condition : isnotnull(web_site_sk#89) - -(89) ColumnarToRow [codegen id : 26] -Input [2]: [web_site_sk#89, web_site_id#90] - -(90) BroadcastExchange -Input [2]: [web_site_sk#89, web_site_id#90] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] - -(91) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_web_site_sk#76] -Right keys [1]: [web_site_sk#89] -Join type: Inner -Join condition: None - -(92) Project [codegen id : 29] -Output [7]: [ws_item_sk#75, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] -Input [9]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_sk#89, web_site_id#90] - -(93) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#91] - -(94) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_item_sk#75] -Right keys [1]: [i_item_sk#91] -Join type: Inner -Join condition: None - -(95) Project [codegen id : 29] -Output [6]: [ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] -Input [8]: [ws_item_sk#75, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90, i_item_sk#91] - -(96) ReusedExchange [Reuses operator id: 34] -Output [1]: [p_promo_sk#92] - -(97) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_promo_sk#77] -Right keys [1]: [p_promo_sk#92] -Join type: Inner -Join condition: None - -(98) Project [codegen id : 29] -Output [5]: [ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] -Input [7]: [ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90, p_promo_sk#92] - -(99) HashAggregate [codegen id : 29] -Input [5]: [ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] -Keys [1]: [web_site_id#90] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#79)), partial_sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))] -Aggregate Attributes [5]: [sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] -Results [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] - -(100) Exchange -Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -Arguments: hashpartitioning(web_site_id#90, 5), ENSURE_REQUIREMENTS, [plan_id=14] - -(101) HashAggregate [codegen id : 30] -Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -Keys [1]: [web_site_id#90] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#79)), sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00)), sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#79))#103, sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00))#104, sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))#105] -Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#79))#103,17,2) AS sales#106, sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00))#104 AS returns#107, sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))#105 AS profit#108, web channel AS channel#109, concat(web_site, web_site_id#90) AS id#110] - -(102) Union - -(103) Expand [codegen id : 31] -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#111, id#112, spark_grouping_id#113] - -(104) HashAggregate [codegen id : 31] -Input [6]: [sales#34, returns#35, profit#36, channel#111, id#112, spark_grouping_id#113] -Keys [3]: [channel#111, id#112, spark_grouping_id#113] -Functions [3]: [partial_sum(sales#34), partial_sum(returns#35), partial_sum(profit#36)] -Aggregate Attributes [6]: [sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] -Results [9]: [channel#111, id#112, spark_grouping_id#113, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] - -(105) Exchange -Input [9]: [channel#111, id#112, spark_grouping_id#113, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] -Arguments: hashpartitioning(channel#111, id#112, spark_grouping_id#113, 5), ENSURE_REQUIREMENTS, [plan_id=15] - -(106) HashAggregate [codegen id : 32] -Input [9]: [channel#111, id#112, spark_grouping_id#113, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] -Keys [3]: [channel#111, id#112, spark_grouping_id#113] -Functions [3]: [sum(sales#34), sum(returns#35), sum(profit#36)] -Aggregate Attributes [3]: [sum(sales#34)#126, sum(returns#35)#127, sum(profit#36)#128] -Results [5]: [channel#111, id#112, sum(sales#34)#126 AS sales#129, sum(returns#35)#127 AS returns#130, sum(profit#36)#128 AS profit#131] - -(107) TakeOrderedAndProject -Input [5]: [channel#111, id#112, sales#129, returns#130, profit#131] -Arguments: 100, [channel#111 ASC NULLS FIRST, id#112 ASC NULLS FIRST], [channel#111, id#112, sales#129, returns#130, profit#131] +(87) CometFilter +Input [2]: [web_site_sk#90, web_site_id#91] +Condition : isnotnull(web_site_sk#90) + +(88) CometBroadcastExchange +Input [2]: [web_site_sk#90, web_site_id#91] +Arguments: [web_site_sk#90, web_site_id#91] + +(89) CometBroadcastHashJoin +Left output [7]: [ws_item_sk#76, ws_web_site_sk#77, ws_promo_sk#78, ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87] +Right output [2]: [web_site_sk#90, web_site_id#91] +Arguments: [ws_web_site_sk#77], [web_site_sk#90], Inner, BuildRight + +(90) CometProject +Input [9]: [ws_item_sk#76, ws_web_site_sk#77, ws_promo_sk#78, ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87, web_site_sk#90, web_site_id#91] +Arguments: [ws_item_sk#76, ws_promo_sk#78, ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87, web_site_id#91], [ws_item_sk#76, ws_promo_sk#78, ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87, web_site_id#91] + +(91) ReusedExchange [Reuses operator id: 26] +Output [1]: [i_item_sk#92] + +(92) CometBroadcastHashJoin +Left output [7]: [ws_item_sk#76, ws_promo_sk#78, ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87, web_site_id#91] +Right output [1]: [i_item_sk#92] +Arguments: [ws_item_sk#76], [i_item_sk#92], Inner, BuildRight + +(93) CometProject +Input [8]: [ws_item_sk#76, ws_promo_sk#78, ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87, web_site_id#91, i_item_sk#92] +Arguments: [ws_promo_sk#78, ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87, web_site_id#91], [ws_promo_sk#78, ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87, web_site_id#91] + +(94) ReusedExchange [Reuses operator id: 32] +Output [1]: [p_promo_sk#93] + +(95) CometBroadcastHashJoin +Left output [6]: [ws_promo_sk#78, ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87, web_site_id#91] +Right output [1]: [p_promo_sk#93] +Arguments: [ws_promo_sk#78], [p_promo_sk#93], Inner, BuildRight + +(96) CometProject +Input [7]: [ws_promo_sk#78, ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87, web_site_id#91, p_promo_sk#93] +Arguments: [ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87, web_site_id#91], [ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87, web_site_id#91] + +(97) ColumnarToRow [codegen id : 5] +Input [5]: [ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87, web_site_id#91] + +(98) HashAggregate [codegen id : 5] +Input [5]: [ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87, web_site_id#91] +Keys [1]: [web_site_id#91] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#80)), partial_sum(coalesce(cast(wr_return_amt#86 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#81 - coalesce(cast(wr_net_loss#87 as decimal(12,2)), 0.00)))] +Aggregate Attributes [5]: [sum#94, sum#95, isEmpty#96, sum#97, isEmpty#98] +Results [6]: [web_site_id#91, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] + +(99) RowToColumnar +Input [6]: [web_site_id#91, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] + +(100) CometColumnarExchange +Input [6]: [web_site_id#91, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] +Arguments: hashpartitioning(web_site_id#91, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(101) ColumnarToRow [codegen id : 6] +Input [6]: [web_site_id#91, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] + +(102) HashAggregate [codegen id : 6] +Input [6]: [web_site_id#91, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] +Keys [1]: [web_site_id#91] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#80)), sum(coalesce(cast(wr_return_amt#86 as decimal(12,2)), 0.00)), sum((ws_net_profit#81 - coalesce(cast(wr_net_loss#87 as decimal(12,2)), 0.00)))] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#80))#104, sum(coalesce(cast(wr_return_amt#86 as decimal(12,2)), 0.00))#105, sum((ws_net_profit#81 - coalesce(cast(wr_net_loss#87 as decimal(12,2)), 0.00)))#106] +Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#80))#104,17,2) AS sales#107, sum(coalesce(cast(wr_return_amt#86 as decimal(12,2)), 0.00))#105 AS returns#108, sum((ws_net_profit#81 - coalesce(cast(wr_net_loss#87 as decimal(12,2)), 0.00)))#106 AS profit#109, web channel AS channel#110, concat(web_site, web_site_id#91) AS id#111] + +(103) Union + +(104) Expand [codegen id : 7] +Input [5]: [sales#35, returns#36, profit#37, channel#38, id#39] +Arguments: [[sales#35, returns#36, profit#37, channel#38, id#39, 0], [sales#35, returns#36, profit#37, channel#38, null, 1], [sales#35, returns#36, profit#37, null, null, 3]], [sales#35, returns#36, profit#37, channel#112, id#113, spark_grouping_id#114] + +(105) HashAggregate [codegen id : 7] +Input [6]: [sales#35, returns#36, profit#37, channel#112, id#113, spark_grouping_id#114] +Keys [3]: [channel#112, id#113, spark_grouping_id#114] +Functions [3]: [partial_sum(sales#35), partial_sum(returns#36), partial_sum(profit#37)] +Aggregate Attributes [6]: [sum#115, isEmpty#116, sum#117, isEmpty#118, sum#119, isEmpty#120] +Results [9]: [channel#112, id#113, spark_grouping_id#114, sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] + +(106) RowToColumnar +Input [9]: [channel#112, id#113, spark_grouping_id#114, sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] + +(107) CometColumnarExchange +Input [9]: [channel#112, id#113, spark_grouping_id#114, sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] +Arguments: hashpartitioning(channel#112, id#113, spark_grouping_id#114, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(108) ColumnarToRow [codegen id : 8] +Input [9]: [channel#112, id#113, spark_grouping_id#114, sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] + +(109) HashAggregate [codegen id : 8] +Input [9]: [channel#112, id#113, spark_grouping_id#114, sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] +Keys [3]: [channel#112, id#113, spark_grouping_id#114] +Functions [3]: [sum(sales#35), sum(returns#36), sum(profit#37)] +Aggregate Attributes [3]: [sum(sales#35)#127, sum(returns#36)#128, sum(profit#37)#129] +Results [5]: [channel#112, id#113, sum(sales#35)#127 AS sales#130, sum(returns#36)#128 AS returns#131, sum(profit#37)#129 AS profit#132] + +(110) TakeOrderedAndProject +Input [5]: [channel#112, id#113, sales#130, returns#131, profit#132] +Arguments: 100, [channel#112 ASC NULLS FIRST, id#113 ASC NULLS FIRST], [channel#112, id#113, sales#130, returns#131, profit#132] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (112) -+- * ColumnarToRow (111) - +- CometProject (110) - +- CometFilter (109) - +- CometScan parquet spark_catalog.default.date_dim (108) +BroadcastExchange (115) ++- * ColumnarToRow (114) + +- CometProject (113) + +- CometFilter (112) + +- CometScan parquet spark_catalog.default.date_dim (111) -(108) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#132] +(111) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#15] 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-22), IsNotNull(d_date_sk)] ReadSchema: struct -(109) CometFilter -Input [2]: [d_date_sk#14, d_date#132] -Condition : (((isnotnull(d_date#132) AND (d_date#132 >= 2000-08-23)) AND (d_date#132 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) +(112) CometFilter +Input [2]: [d_date_sk#14, d_date#15] +Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#15 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) -(110) CometProject -Input [2]: [d_date_sk#14, d_date#132] +(113) CometProject +Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(111) ColumnarToRow [codegen id : 1] +(114) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(112) BroadcastExchange +(115) BroadcastExchange Input [1]: [d_date_sk#14] -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=11] -Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 41 Hosting Expression = cs_sold_date_sk#46 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 71 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 72 Hosting Expression = ws_sold_date_sk#82 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/simplified.txt index cdc8dc486..bb5fbbbcc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/simplified.txt @@ -1,37 +1,37 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (32) + 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 (31) - 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 (10) - HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [s_store_id] #2 - WholeStageCodegen (9) - HashAggregate [s_store_id,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - SortMergeJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - WholeStageCodegen (2) - Sort [ss_item_sk,ss_ticket_number] - InputAdapter - Exchange [ss_item_sk,ss_ticket_number] #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometColumnarExchange [channel,id,spark_grouping_id] #1 + RowToColumnar + 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 (2) + HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [s_store_id] #2 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [s_store_id,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] + CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] + CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] + CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk,ss_ticket_number] #3 CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -42,141 +42,95 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometProject [d_date_sk] CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - WholeStageCodegen (4) - Sort [sr_item_sk,sr_ticket_number] - InputAdapter - Exchange [sr_item_sk,sr_ticket_number] #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter + CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometColumnarExchange [sr_item_sk,sr_ticket_number] #5 CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - 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 (7) - ColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - WholeStageCodegen (20) - HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [cp_catalog_page_id] #9 - WholeStageCodegen (19) - HashAggregate [cp_catalog_page_id,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_catalog_page_sk,cp_catalog_page_sk] - Project [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] - InputAdapter - WholeStageCodegen (12) - Sort [cs_item_sk,cs_order_number] - InputAdapter - Exchange [cs_item_sk,cs_order_number] #10 - WholeStageCodegen (11) - ColumnarToRow - InputAdapter + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [s_store_sk,s_store_id] #7 + CometFilter [s_store_sk,s_store_id] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange [i_item_sk] #8 + CometProject [i_item_sk] + CometFilter [i_item_sk,i_current_price] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] + CometBroadcastExchange [p_promo_sk] #9 + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] + CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] + WholeStageCodegen (4) + HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [cp_catalog_page_id] #10 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [cp_catalog_page_id,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] + CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] + CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] + CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometColumnarExchange [cs_item_sk,cs_order_number] #11 CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - WholeStageCodegen (14) - Sort [cr_item_sk,cr_order_number] - InputAdapter - Exchange [cr_item_sk,cr_order_number] #11 - WholeStageCodegen (13) - ColumnarToRow - InputAdapter + CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometColumnarExchange [cr_item_sk,cr_order_number] #12 CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - InputAdapter - ReusedExchange [i_item_sk] #7 - InputAdapter - ReusedExchange [p_promo_sk] #8 - WholeStageCodegen (30) - HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [web_site_id] #13 - WholeStageCodegen (29) - HashAggregate [web_site_id,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_promo_sk,p_promo_sk] - Project [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - SortMergeJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] - InputAdapter - WholeStageCodegen (22) - Sort [ws_item_sk,ws_order_number] - InputAdapter - Exchange [ws_item_sk,ws_order_number] #14 - WholeStageCodegen (21) - ColumnarToRow - InputAdapter + ReusedExchange [d_date_sk] #6 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #13 + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + ReusedExchange [i_item_sk] #8 + ReusedExchange [p_promo_sk] #9 + WholeStageCodegen (6) + HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [web_site_id] #14 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [web_site_id,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] + CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] + CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] + CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometColumnarExchange [ws_item_sk,ws_order_number] #15 CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - WholeStageCodegen (24) - Sort [wr_item_sk,wr_order_number] - InputAdapter - Exchange [wr_item_sk,wr_order_number] #15 - WholeStageCodegen (23) - ColumnarToRow - InputAdapter + CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometColumnarExchange [wr_item_sk,wr_order_number] #16 CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #16 - WholeStageCodegen (26) - ColumnarToRow - InputAdapter - CometFilter [web_site_sk,web_site_id] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - InputAdapter - ReusedExchange [i_item_sk] #7 - InputAdapter - ReusedExchange [p_promo_sk] #8 + ReusedExchange [d_date_sk] #6 + CometBroadcastExchange [web_site_sk,web_site_id] #17 + CometFilter [web_site_sk,web_site_id] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + ReusedExchange [i_item_sk] #8 + ReusedExchange [p_promo_sk] #9 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 b266f4a24..9c8482d6e 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,54 +1,60 @@ == Physical Plan == -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) +TakeOrderedAndProject (56) ++- * Project (55) + +- * BroadcastHashJoin Inner BuildRight (54) + :- * Project (49) + : +- * BroadcastHashJoin Inner BuildRight (48) + : :- * Project (43) + : : +- * BroadcastHashJoin Inner BuildRight (42) + : : :- * Filter (20) + : : : +- * HashAggregate (19) + : : : +- * ColumnarToRow (18) + : : : +- CometColumnarExchange (17) + : : : +- RowToColumnar (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 (41) + : : +- * Filter (40) + : : +- * HashAggregate (39) + : : +- * ColumnarToRow (38) + : : +- CometColumnarExchange (37) + : : +- RowToColumnar (36) + : : +- * HashAggregate (35) + : : +- * HashAggregate (34) + : : +- * ColumnarToRow (33) + : : +- CometColumnarExchange (32) + : : +- RowToColumnar (31) + : : +- * HashAggregate (30) + : : +- * ColumnarToRow (29) + : : +- CometProject (28) + : : +- CometBroadcastHashJoin (27) + : : :- CometProject (25) + : : : +- CometBroadcastHashJoin (24) + : : : :- CometFilter (22) + : : : : +- CometScan parquet spark_catalog.default.catalog_returns (21) + : : : +- ReusedExchange (23) + : : +- ReusedExchange (26) + : +- BroadcastExchange (47) + : +- * ColumnarToRow (46) + : +- CometFilter (45) + : +- CometScan parquet spark_catalog.default.customer (44) + +- BroadcastExchange (53) + +- * ColumnarToRow (52) + +- CometFilter (51) + +- CometScan parquet spark_catalog.default.customer_address (50) (1) Scan parquet spark_catalog.default.catalog_returns @@ -125,22 +131,28 @@ Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] Aggregate Attributes [1]: [sum#10] Results [3]: [cr_returning_customer_sk#1, ca_state#9, sum#11] -(16) Exchange +(16) RowToColumnar 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] -(17) HashAggregate [codegen id : 7] +(17) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(18) ColumnarToRow [codegen id : 7] +Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#11] + +(19) 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))#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] -(18) Filter [codegen id : 7] +(20) Filter [codegen id : 7] Input [3]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15] Condition : isnotnull(ctr_total_return#15) -(19) Scan parquet spark_catalog.default.catalog_returns +(21) 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 [] @@ -148,183 +160,195 @@ PartitionFilters: [isnotnull(cr_returned_date_sk#19), dynamicpruningexpression(c PushedFilters: [IsNotNull(cr_returning_addr_sk)] ReadSchema: struct -(20) CometFilter +(22) 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] +(23) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#21] -(22) CometBroadcastHashJoin +(24) 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 +(25) 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] +(26) ReusedExchange [Reuses operator id: 11] Output [2]: [ca_address_sk#22, ca_state#23] -(25) CometBroadcastHashJoin +(27) 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 +(28) 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] +(29) 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] +(30) 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 +(31) RowToColumnar 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] +(32) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] + +(33) ColumnarToRow [codegen id : 3] +Input [3]: [cr_returning_customer_sk#16, ca_state#23, sum#25] + +(34) 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] +(35) 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 +(36) RowToColumnar +Input [3]: [ctr_state#26, sum#30, count#31] + +(37) CometColumnarExchange Input [3]: [ctr_state#26, sum#30, count#31] -Arguments: hashpartitioning(ctr_state#26, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(ctr_state#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(33) HashAggregate [codegen id : 4] +(38) ColumnarToRow [codegen id : 4] +Input [3]: [ctr_state#26, sum#30, count#31] + +(39) 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] +(40) 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 +(41) 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] +(42) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ctr_state#14] Right keys [1]: [ctr_state#26] Join type: Inner Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#33) -(37) Project [codegen id : 7] +(43) 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] -(38) Scan parquet spark_catalog.default.customer +(44) 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 -(39) CometFilter +(45) 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)) -(40) ColumnarToRow [codegen id : 5] +(46) 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] -(41) BroadcastExchange +(47) 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] -(42) BroadcastHashJoin [codegen id : 7] +(48) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ctr_customer_sk#13] Right keys [1]: [c_customer_sk#34] Join type: Inner Join condition: None -(43) Project [codegen id : 7] +(49) 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] -(44) Scan parquet spark_catalog.default.customer_address +(50) 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 -(45) CometFilter +(51) 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)) -(46) ColumnarToRow [codegen id : 6] +(52) 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] -(47) BroadcastExchange +(53) 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] -(48) BroadcastHashJoin [codegen id : 7] +(54) 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 -(49) Project [codegen id : 7] +(55) 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] -(50) TakeOrderedAndProject +(56) 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 (55) -+- * ColumnarToRow (54) - +- CometProject (53) - +- CometFilter (52) - +- CometScan parquet spark_catalog.default.date_dim (51) +BroadcastExchange (61) ++- * ColumnarToRow (60) + +- CometProject (59) + +- CometFilter (58) + +- CometScan parquet spark_catalog.default.date_dim (57) -(51) Scan parquet spark_catalog.default.date_dim +(57) 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 -(52) CometFilter +(58) 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)) -(53) CometProject +(59) CometProject Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(54) ColumnarToRow [codegen id : 1] +(60) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(55) BroadcastExchange +(61) 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 = 19 Hosting Expression = cr_returned_date_sk#19 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 21 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 991c00de9..da54f3a03 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 @@ -8,58 +8,64 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] Filter [ctr_total_return] HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_customer_sk,ctr_state,ctr_total_return,sum] - InputAdapter - Exchange [cr_returning_customer_sk,ca_state] #1 - WholeStageCodegen (1) - HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] - CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] - CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [ca_address_sk,ca_state] #4 - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + ColumnarToRow + InputAdapter + CometColumnarExchange [cr_returning_customer_sk,ca_state] #1 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] + CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] + CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [ca_address_sk,ca_state] #4 + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter 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] #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] #7 - WholeStageCodegen (2) - HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] - CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] - CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_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 - ReusedExchange [d_date_sk] #3 - ReusedExchange [ca_address_sk,ca_state] #4 + ColumnarToRow + InputAdapter + CometColumnarExchange [ctr_state] #6 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [cr_returning_customer_sk,ca_state] #7 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] + CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] + CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_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 + ReusedExchange [d_date_sk] #3 + ReusedExchange [ca_address_sk,ca_state] #4 InputAdapter BroadcastExchange #8 WholeStageCodegen (5) 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 00e38f694..2519fbc97 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,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (26) -+- * HashAggregate (25) - +- Exchange (24) - +- * ColumnarToRow (23) +* ColumnarToRow (26) ++- CometTakeOrderedAndProject (25) + +- CometHashAggregate (24) + +- CometColumnarExchange (23) +- CometHashAggregate (22) +- CometProject (21) +- CometBroadcastHashJoin (20) @@ -132,23 +132,21 @@ 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: [] -(23) ColumnarToRow [codegen id : 1] +(23) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(24) 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] -(26) TakeOrderedAndProject +(25) CometTakeOrderedAndProject +Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#2 ASC NULLS FIRST], output=[i_item_id#2,i_item_desc#3,i_current_price#4]), [i_item_id#2, i_item_desc#3, i_current_price#4], 100, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4] + +(26) ColumnarToRow [codegen id : 1] 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 ===== 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 71a1c8e17..869073de1 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,38 +1,36 @@ -TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - 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 (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,i_item_id,i_item_desc,i_current_price,ss_item_sk] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2 - CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,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,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] - CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 - CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_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_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometProject [ss_item_sk] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] + CometHashAggregate [i_item_id,i_item_desc,i_current_price] + CometColumnarExchange [i_item_id,i_item_desc,i_current_price] #1 + CometHashAggregate [i_item_id,i_item_desc,i_current_price] + CometProject [i_item_id,i_item_desc,i_current_price] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,ss_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2 + CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,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,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] + CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] + CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 + CometProject [inv_item_sk,inv_date_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_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_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [ss_item_sk] + CometFilter [ss_item_sk,ss_sold_date_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 d9d319813..49ee36742 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,54 +1,60 @@ == Physical Plan == -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) +TakeOrderedAndProject (56) ++- * Project (55) + +- * BroadcastHashJoin Inner BuildRight (54) + :- * Project (38) + : +- * BroadcastHashJoin Inner BuildRight (37) + : :- * HashAggregate (21) + : : +- * ColumnarToRow (20) + : : +- CometColumnarExchange (19) + : : +- RowToColumnar (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 (36) + : +- * HashAggregate (35) + : +- * ColumnarToRow (34) + : +- CometColumnarExchange (33) + : +- RowToColumnar (32) + : +- * HashAggregate (31) + : +- * ColumnarToRow (30) + : +- CometProject (29) + : +- CometBroadcastHashJoin (28) + : :- CometProject (26) + : : +- CometBroadcastHashJoin (25) + : : :- CometFilter (23) + : : : +- CometScan parquet spark_catalog.default.catalog_returns (22) + : : +- ReusedExchange (24) + : +- ReusedExchange (27) + +- BroadcastExchange (53) + +- * HashAggregate (52) + +- * ColumnarToRow (51) + +- CometColumnarExchange (50) + +- RowToColumnar (49) + +- * HashAggregate (48) + +- * ColumnarToRow (47) + +- CometProject (46) + +- CometBroadcastHashJoin (45) + :- CometProject (43) + : +- CometBroadcastHashJoin (42) + : :- CometFilter (40) + : : +- CometScan parquet spark_catalog.default.web_returns (39) + : +- ReusedExchange (41) + +- ReusedExchange (44) (1) Scan parquet spark_catalog.default.store_returns @@ -98,7 +104,7 @@ ReadSchema: struct Input [2]: [d_date_sk#7, d_date#8] Condition : isnotnull(d_date_sk#7) -(10) ReusedExchange [Reuses operator id: 60] +(10) ReusedExchange [Reuses operator id: 66] Output [1]: [d_date#9] (11) CometBroadcastHashJoin @@ -133,18 +139,24 @@ Functions [1]: [partial_sum(sr_return_quantity#2)] Aggregate Attributes [1]: [sum#10] Results [2]: [i_item_id#6, sum#11] -(18) Exchange +(18) RowToColumnar Input [2]: [i_item_id#6, sum#11] -Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(19) HashAggregate [codegen id : 6] +(19) CometColumnarExchange +Input [2]: [i_item_id#6, sum#11] +Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(20) ColumnarToRow [codegen id : 6] +Input [2]: [i_item_id#6, sum#11] + +(21) 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)#12] Results [2]: [i_item_id#6 AS item_id#13, sum(sr_return_quantity#2)#12 AS sr_item_qty#14] -(20) Scan parquet spark_catalog.default.catalog_returns +(22) 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 [] @@ -152,70 +164,76 @@ PartitionFilters: [isnotnull(cr_returned_date_sk#17), dynamicpruningexpression(c PushedFilters: [IsNotNull(cr_item_sk)] ReadSchema: struct -(21) CometFilter +(23) 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] +(24) ReusedExchange [Reuses operator id: 5] Output [2]: [i_item_sk#19, i_item_id#20] -(23) CometBroadcastHashJoin +(25) 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 +(26) 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] +(27) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#21] -(26) CometBroadcastHashJoin +(28) 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 +(29) 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] +(30) ColumnarToRow [codegen id : 2] Input [2]: [cr_return_quantity#16, i_item_id#20] -(29) HashAggregate [codegen id : 2] +(31) 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 +(32) RowToColumnar +Input [2]: [i_item_id#20, sum#23] + +(33) CometColumnarExchange +Input [2]: [i_item_id#20, sum#23] +Arguments: hashpartitioning(i_item_id#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(34) ColumnarToRow [codegen id : 3] 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] +(35) 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 +(36) 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] +(37) BroadcastHashJoin [codegen id : 6] Left keys [1]: [item_id#13] Right keys [1]: [item_id#25] Join type: Inner Join condition: None -(34) Project [codegen id : 6] +(38) 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] -(35) Scan parquet spark_catalog.default.web_returns +(39) 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 [] @@ -223,159 +241,165 @@ PartitionFilters: [isnotnull(wr_returned_date_sk#29), dynamicpruningexpression(w PushedFilters: [IsNotNull(wr_item_sk)] ReadSchema: struct -(36) CometFilter +(40) 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] +(41) ReusedExchange [Reuses operator id: 5] Output [2]: [i_item_sk#31, i_item_id#32] -(38) CometBroadcastHashJoin +(42) 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 +(43) 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] +(44) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#33] -(41) CometBroadcastHashJoin +(45) 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 +(46) 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] +(47) ColumnarToRow [codegen id : 4] Input [2]: [wr_return_quantity#28, i_item_id#32] -(44) HashAggregate [codegen id : 4] +(48) 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 +(49) RowToColumnar +Input [2]: [i_item_id#32, sum#35] + +(50) CometColumnarExchange Input [2]: [i_item_id#32, sum#35] -Arguments: hashpartitioning(i_item_id#32, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: hashpartitioning(i_item_id#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(46) HashAggregate [codegen id : 5] +(51) ColumnarToRow [codegen id : 5] +Input [2]: [i_item_id#32, sum#35] + +(52) 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 +(53) 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] +(54) BroadcastHashJoin [codegen id : 6] Left keys [1]: [item_id#13] Right keys [1]: [item_id#37] Join type: Inner Join condition: None -(49) Project [codegen id : 6] +(55) 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] -(50) TakeOrderedAndProject +(56) 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 (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 +BroadcastExchange (70) ++- * ColumnarToRow (69) + +- CometProject (68) + +- CometBroadcastHashJoin (67) + :- CometFilter (58) + : +- CometScan parquet spark_catalog.default.date_dim (57) + +- CometBroadcastExchange (66) + +- CometProject (65) + +- CometBroadcastHashJoin (64) + :- CometScan parquet spark_catalog.default.date_dim (59) + +- CometBroadcastExchange (63) + +- CometProject (62) + +- CometFilter (61) + +- CometScan parquet spark_catalog.default.date_dim (60) + + +(57) 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 -(52) CometFilter +(58) CometFilter Input [2]: [d_date_sk#7, d_date#8] Condition : isnotnull(d_date_sk#7) -(53) Scan parquet spark_catalog.default.date_dim +(59) 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 -(54) Scan parquet spark_catalog.default.date_dim +(60) 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 -(55) CometFilter +(61) CometFilter Input [2]: [d_date#44, d_week_seq#45] Condition : d_date#44 IN (2000-06-30,2000-09-27,2000-11-17) -(56) CometProject +(62) CometProject Input [2]: [d_date#44, d_week_seq#45] Arguments: [d_week_seq#45], [d_week_seq#45] -(57) CometBroadcastExchange +(63) CometBroadcastExchange Input [1]: [d_week_seq#45] Arguments: [d_week_seq#45] -(58) CometBroadcastHashJoin +(64) 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 -(59) CometProject +(65) CometProject Input [2]: [d_date#9, d_week_seq#43] Arguments: [d_date#9], [d_date#9] -(60) CometBroadcastExchange +(66) CometBroadcastExchange Input [1]: [d_date#9] Arguments: [d_date#9] -(61) CometBroadcastHashJoin +(67) 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 -(62) CometProject +(68) CometProject Input [2]: [d_date_sk#7, d_date#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(63) ColumnarToRow [codegen id : 1] +(69) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(64) BroadcastExchange +(70) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 20 Hosting Expression = cr_returned_date_sk#17 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 22 Hosting Expression = cr_returned_date_sk#17 IN dynamicpruning#4 -Subquery:3 Hosting operator id = 35 Hosting Expression = wr_returned_date_sk#29 IN dynamicpruning#4 +Subquery:3 Hosting operator id = 39 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 8cd3f3f45..505ef9b5c 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 @@ -5,79 +5,85 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty Project [item_id,sr_item_qty,cr_item_qty] BroadcastHashJoin [item_id,item_id] HashAggregate [i_item_id,sum] [sum(sr_return_quantity),item_id,sr_item_qty,sum] - InputAdapter - Exchange [i_item_id] #1 - WholeStageCodegen (1) - HashAggregate [i_item_id,sr_return_quantity] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [sr_return_quantity,i_item_id] - CometBroadcastHashJoin [sr_return_quantity,sr_returned_date_sk,i_item_id,d_date_sk] - CometProject [sr_return_quantity,sr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [sr_item_sk,sr_return_quantity,sr_returned_date_sk,i_item_sk,i_item_id] - CometFilter [sr_item_sk,sr_return_quantity,sr_returned_date_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 (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] - CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] - CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] - CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [i_item_sk,i_item_id] #5 - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date] #3 + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #1 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [i_item_id,sr_return_quantity] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [sr_return_quantity,i_item_id] + CometBroadcastHashJoin [sr_return_quantity,sr_returned_date_sk,i_item_id,d_date_sk] + CometProject [sr_return_quantity,sr_returned_date_sk,i_item_id] + CometBroadcastHashJoin [sr_item_sk,sr_return_quantity,sr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [sr_item_sk,sr_return_quantity,sr_returned_date_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 (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date] #3 + CometProject [d_date] + CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometBroadcastExchange [d_week_seq] #4 + CometProject [d_week_seq] + CometFilter [d_date,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometBroadcastExchange [i_item_sk,i_item_id] #5 + CometFilter [i_item_sk,i_item_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + ReusedExchange [d_date] #3 InputAdapter BroadcastExchange #7 WholeStageCodegen (3) HashAggregate [i_item_id,sum] [sum(cr_return_quantity),item_id,cr_item_qty,sum] - InputAdapter - Exchange [i_item_id] #8 - WholeStageCodegen (2) - HashAggregate [i_item_id,cr_return_quantity] [sum,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #8 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [i_item_id,cr_return_quantity] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [cr_return_quantity,i_item_id] + CometBroadcastHashJoin [cr_return_quantity,cr_returned_date_sk,i_item_id,d_date_sk] + CometProject [cr_return_quantity,cr_returned_date_sk,i_item_id] + CometBroadcastHashJoin [cr_item_sk,cr_return_quantity,cr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [cr_item_sk,cr_return_quantity,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [i_item_sk,i_item_id] #5 + ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (5) + HashAggregate [i_item_id,sum] [sum(wr_return_quantity),item_id,wr_item_qty,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #10 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [i_item_id,wr_return_quantity] [sum,sum] ColumnarToRow InputAdapter - CometProject [cr_return_quantity,i_item_id] - CometBroadcastHashJoin [cr_return_quantity,cr_returned_date_sk,i_item_id,d_date_sk] - CometProject [cr_return_quantity,cr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [cr_item_sk,cr_return_quantity,cr_returned_date_sk,i_item_sk,i_item_id] - CometFilter [cr_item_sk,cr_return_quantity,cr_returned_date_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] + CometProject [wr_return_quantity,i_item_id] + CometBroadcastHashJoin [wr_return_quantity,wr_returned_date_sk,i_item_id,d_date_sk] + CometProject [wr_return_quantity,wr_returned_date_sk,i_item_id] + CometBroadcastHashJoin [wr_item_sk,wr_return_quantity,wr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [wr_item_sk,wr_return_quantity,wr_returned_date_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #5 ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (5) - HashAggregate [i_item_id,sum] [sum(wr_return_quantity),item_id,wr_item_qty,sum] - InputAdapter - Exchange [i_item_id] #10 - WholeStageCodegen (4) - HashAggregate [i_item_id,wr_return_quantity] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [wr_return_quantity,i_item_id] - CometBroadcastHashJoin [wr_return_quantity,wr_returned_date_sk,i_item_id,d_date_sk] - CometProject [wr_return_quantity,wr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [wr_item_sk,wr_return_quantity,wr_returned_date_sk,i_item_sk,i_item_id] - CometFilter [wr_item_sk,wr_return_quantity,wr_returned_date_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - 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/q85/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/explain.txt index c8becd0e9..d02f4cc80 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,49 +1,51 @@ == Physical Plan == -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) +TakeOrderedAndProject (47) ++- * HashAggregate (46) + +- * ColumnarToRow (45) + +- CometColumnarExchange (44) + +- RowToColumnar (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 @@ -248,50 +250,56 @@ Functions [3]: [partial_avg(ws_quantity#4), partial_avg(UnscaledValue(wr_refunde 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] -(43) Exchange +(43) RowToColumnar 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] -(44) HashAggregate [codegen id : 2] +(44) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(45) ColumnarToRow [codegen id : 2] +Input [7]: [r_reason_desc#31, sum#38, count#39, sum#40, count#41, sum#42, count#43] + +(46) 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)#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] -(45) TakeOrderedAndProject +(47) 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 (50) -+- * ColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan parquet spark_catalog.default.date_dim (46) +BroadcastExchange (52) ++- * ColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) -(46) Scan parquet spark_catalog.default.date_dim +(48) 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 -(47) CometFilter +(49) 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)) -(48) CometProject +(50) CometProject Input [2]: [d_date_sk#28, d_year#29] Arguments: [d_date_sk#28], [d_date_sk#28] -(49) ColumnarToRow [codegen id : 1] +(51) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#28] -(50) BroadcastExchange +(52) BroadcastExchange Input [1]: [d_date_sk#28] 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 0d350bd59..af70cc827 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,57 +1,59 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee)] 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 (1) - HashAggregate [r_reason_desc,ws_quantity,wr_refunded_cash,wr_fee] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - ColumnarToRow - InputAdapter - CometProject [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] - CometBroadcastHashJoin [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash,r_reason_sk,r_reason_desc] - CometProject [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash,d_date_sk] - CometProject [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash,ca_address_sk,ca_state] - CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [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,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 [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,cd_demo_sk,cd_marital_status,cd_education_status] - 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,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,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_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,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] - CometBroadcastExchange [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] #2 - CometFilter [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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_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] - 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] - CometBroadcastExchange [wp_web_page_sk] #4 - CometFilter [wp_web_page_sk] - CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #5 + ColumnarToRow + InputAdapter + CometColumnarExchange [r_reason_desc] #1 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometProject [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] + CometBroadcastHashJoin [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash,r_reason_sk,r_reason_desc] + CometProject [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash] + CometBroadcastHashJoin [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash,d_date_sk] + CometProject [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash] + CometBroadcastHashJoin [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash,ca_address_sk,ca_state] + CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash] + CometBroadcastHashJoin [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,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 [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,cd_demo_sk,cd_marital_status,cd_education_status] + 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,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,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_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,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] + CometBroadcastExchange [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] #2 + CometFilter [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + 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_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] + 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] + CometBroadcastExchange [wp_web_page_sk] #4 + CometFilter [wp_web_page_sk] + CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #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] + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #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 [cd_demo_sk,cd_marital_status,cd_education_status] #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 [ca_address_sk,ca_state] #7 - CometProject [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #8 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [r_reason_sk,r_reason_desc] #9 - CometFilter [r_reason_sk,r_reason_desc] - CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] + CometBroadcastExchange [ca_address_sk,ca_state] #7 + CometProject [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + CometBroadcastExchange [d_date_sk] #8 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [r_reason_sk,r_reason_desc] #9 + CometFilter [r_reason_sk,r_reason_desc] + 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 1c9688ca2..4300d3efc 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,27 +1,31 @@ == Physical Plan == -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) +TakeOrderedAndProject (27) ++- * Project (26) + +- Window (25) + +- * ColumnarToRow (24) + +- CometSort (23) + +- CometColumnarExchange (22) + +- RowToColumnar (21) + +- * HashAggregate (20) + +- * ColumnarToRow (19) + +- CometColumnarExchange (18) + +- RowToColumnar (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 @@ -102,66 +106,78 @@ Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] Aggregate Attributes [1]: [sum#13] Results [4]: [i_category#10, i_class#11, spark_grouping_id#12, sum#14] -(17) Exchange +(17) RowToColumnar 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] -(18) HashAggregate [codegen id : 2] +(18) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(19) ColumnarToRow [codegen id : 2] +Input [4]: [i_category#10, i_class#11, spark_grouping_id#12, sum#14] + +(20) 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))#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] -(19) Exchange +(21) RowToColumnar +Input [7]: [total_sum#16, i_category#10, i_class#11, lochierarchy#17, _w0#18, _w1#19, _w2#20] + +(22) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] + +(23) CometSort 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] +Arguments: [total_sum#16, i_category#10, i_class#11, lochierarchy#17, _w0#18, _w1#19, _w2#20], [_w1#19 ASC NULLS FIRST, _w2#20 ASC NULLS FIRST, _w0#18 DESC NULLS LAST] -(20) Sort [codegen id : 3] +(24) ColumnarToRow [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 -(21) Window +(25) 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] -(22) Project [codegen id : 4] +(26) 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] -(23) TakeOrderedAndProject +(27) 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 (28) -+- * ColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan parquet spark_catalog.default.date_dim (24) +BroadcastExchange (32) ++- * ColumnarToRow (31) + +- CometProject (30) + +- CometFilter (29) + +- CometScan parquet spark_catalog.default.date_dim (28) -(24) Scan parquet spark_catalog.default.date_dim +(28) 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 +(29) 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 +(30) 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] +(31) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(28) BroadcastExchange +(32) BroadcastExchange Input [1]: [d_date_sk#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/q86/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt index e4d028f2f..e8897d5fd 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 @@ -4,36 +4,40 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl InputAdapter Window [_w0,_w1,_w2] WholeStageCodegen (3) - Sort [_w1,_w2,_w0] + ColumnarToRow InputAdapter - Exchange [_w1,_w2] #1 - 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 (1) - HashAggregate [i_category,i_class,spark_grouping_id,ws_net_paid] [sum,sum] - 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,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometFilter [i_item_sk,i_class,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] + CometColumnarExchange [_w1,_w2] #1 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [i_category,i_class,spark_grouping_id,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,lochierarchy,_w0,_w1,_w2,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class,spark_grouping_id] #2 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [i_category,i_class,spark_grouping_id,ws_net_paid] [sum,sum] + 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,ws_net_paid,i_item_sk,i_class,i_category] + CometProject [ws_item_sk,ws_net_paid] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometFilter [i_item_sk,i_class,i_category] + 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 0434066e6..d7ea7098e 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,53 +1,55 @@ == Physical Plan == -* 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) +* HashAggregate (51) ++- * ColumnarToRow (50) + +- CometColumnarExchange (49) + +- RowToColumnar (48) + +- * HashAggregate (47) + +- * Project (46) + +- * BroadcastHashJoin LeftAnti BuildRight (45) + :- * BroadcastHashJoin LeftAnti BuildRight (31) + : :- * ColumnarToRow (17) + : : +- CometHashAggregate (16) + : : +- CometColumnarExchange (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) + : +- * ColumnarToRow (29) + : +- CometHashAggregate (28) + : +- CometColumnarExchange (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) + +- * ColumnarToRow (43) + +- CometHashAggregate (42) + +- CometColumnarExchange (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 @@ -119,19 +121,17 @@ 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: [] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(17) HashAggregate [codegen id : 6] +(16) 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#9, c_first_name#8, d_date#5] + +(17) ColumnarToRow [codegen id : 3] +Input [3]: [c_last_name#9, c_first_name#8, d_date#5] (18) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_bill_customer_sk#10, cs_sold_date_sk#11] @@ -174,25 +174,23 @@ 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: [] -(27) ColumnarToRow [codegen id : 2] +(27) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=2] -(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] +(28) 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#17, c_first_name#16, d_date#14] + +(29) ColumnarToRow [codegen id : 1] +Input [3]: [c_last_name#17, c_first_name#16, d_date#14] (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] -(31) BroadcastHashJoin [codegen id : 6] +(31) BroadcastHashJoin [codegen id : 3] 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 @@ -239,46 +237,50 @@ 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: [] -(41) ColumnarToRow [codegen id : 4] +(41) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=4] -(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] - -(43) HashAggregate [codegen id : 5] +(42) 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#25, c_first_name#24, d_date#22] + +(43) ColumnarToRow [codegen id : 2] +Input [3]: [c_last_name#25, c_first_name#24, d_date#22] (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] -(45) BroadcastHashJoin [codegen id : 6] +(45) BroadcastHashJoin [codegen id : 3] 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 -(46) Project [codegen id : 6] +(46) Project [codegen id : 3] Output: [] Input [3]: [c_last_name#9, c_first_name#8, d_date#5] -(47) HashAggregate [codegen id : 6] +(47) HashAggregate [codegen id : 3] Input: [] Keys: [] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#26] Results [1]: [count#27] -(48) Exchange +(48) RowToColumnar +Input [1]: [count#27] + +(49) CometColumnarExchange +Input [1]: [count#27] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(50) ColumnarToRow [codegen id : 4] Input [1]: [count#27] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(49) HashAggregate [codegen id : 7] +(51) HashAggregate [codegen id : 4] Input [1]: [count#27] Keys: [] Functions [1]: [count(1)] @@ -288,32 +290,32 @@ 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 (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 +(52) 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 -(51) CometFilter +(53) 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)) -(52) CometProject +(54) 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] -(53) ColumnarToRow [codegen id : 1] +(55) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#4, d_date#5] -(54) BroadcastExchange +(56) BroadcastExchange Input [2]: [d_date_sk#4, d_date#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] 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 9d667265c..69f33a57a 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,18 +1,18 @@ -WholeStageCodegen (7) +WholeStageCodegen (4) HashAggregate [count] [count(1),count(1),count] - InputAdapter - Exchange #1 - WholeStageCodegen (6) - HashAggregate [count,count] - Project - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometColumnarExchange #1 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [count,count] + Project + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + ColumnarToRow + InputAdapter + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #2 CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] @@ -35,15 +35,13 @@ WholeStageCodegen (7) CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #7 CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] CometBroadcastHashJoin [cs_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] @@ -54,15 +52,13 @@ WholeStageCodegen (7) ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #9 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #9 CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] CometBroadcastHashJoin [ws_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] 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 3f905ebe9..cd9e4817c 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 @@ -6,9 +6,9 @@ : : : :- * BroadcastNestedLoopJoin Inner BuildRight (88) : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (67) : : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (46) -: : : : : : :- * HashAggregate (25) -: : : : : : : +- Exchange (24) -: : : : : : : +- * ColumnarToRow (23) +: : : : : : :- * ColumnarToRow (25) +: : : : : : : +- CometHashAggregate (24) +: : : : : : : +- CometColumnarExchange (23) : : : : : : : +- CometHashAggregate (22) : : : : : : : +- CometProject (21) : : : : : : : +- CometBroadcastHashJoin (20) @@ -32,9 +32,9 @@ : : : : : : : +- CometFilter (17) : : : : : : : +- CometScan parquet spark_catalog.default.store (16) : : : : : : +- BroadcastExchange (45) -: : : : : : +- * HashAggregate (44) -: : : : : : +- Exchange (43) -: : : : : : +- * ColumnarToRow (42) +: : : : : : +- * ColumnarToRow (44) +: : : : : : +- CometHashAggregate (43) +: : : : : : +- CometColumnarExchange (42) : : : : : : +- CometHashAggregate (41) : : : : : : +- CometProject (40) : : : : : : +- CometBroadcastHashJoin (39) @@ -52,9 +52,9 @@ : : : : : : : +- CometScan parquet spark_catalog.default.time_dim (32) : : : : : : +- ReusedExchange (38) : : : : : +- BroadcastExchange (66) -: : : : : +- * HashAggregate (65) -: : : : : +- Exchange (64) -: : : : : +- * ColumnarToRow (63) +: : : : : +- * ColumnarToRow (65) +: : : : : +- CometHashAggregate (64) +: : : : : +- CometColumnarExchange (63) : : : : : +- CometHashAggregate (62) : : : : : +- CometProject (61) : : : : : +- CometBroadcastHashJoin (60) @@ -72,9 +72,9 @@ : : : : : : +- CometScan parquet spark_catalog.default.time_dim (53) : : : : : +- ReusedExchange (59) : : : : +- BroadcastExchange (87) -: : : : +- * HashAggregate (86) -: : : : +- Exchange (85) -: : : : +- * ColumnarToRow (84) +: : : : +- * ColumnarToRow (86) +: : : : +- CometHashAggregate (85) +: : : : +- CometColumnarExchange (84) : : : : +- CometHashAggregate (83) : : : : +- CometProject (82) : : : : +- CometBroadcastHashJoin (81) @@ -92,9 +92,9 @@ : : : : : +- CometScan parquet spark_catalog.default.time_dim (74) : : : : +- ReusedExchange (80) : : : +- BroadcastExchange (108) -: : : +- * HashAggregate (107) -: : : +- Exchange (106) -: : : +- * ColumnarToRow (105) +: : : +- * ColumnarToRow (107) +: : : +- CometHashAggregate (106) +: : : +- CometColumnarExchange (105) : : : +- CometHashAggregate (104) : : : +- CometProject (103) : : : +- CometBroadcastHashJoin (102) @@ -112,9 +112,9 @@ : : : : +- CometScan parquet spark_catalog.default.time_dim (95) : : : +- ReusedExchange (101) : : +- BroadcastExchange (129) -: : +- * HashAggregate (128) -: : +- Exchange (127) -: : +- * ColumnarToRow (126) +: : +- * ColumnarToRow (128) +: : +- CometHashAggregate (127) +: : +- CometColumnarExchange (126) : : +- CometHashAggregate (125) : : +- CometProject (124) : : +- CometBroadcastHashJoin (123) @@ -132,9 +132,9 @@ : : : +- CometScan parquet spark_catalog.default.time_dim (116) : : +- ReusedExchange (122) : +- BroadcastExchange (150) -: +- * HashAggregate (149) -: +- Exchange (148) -: +- * ColumnarToRow (147) +: +- * ColumnarToRow (149) +: +- CometHashAggregate (148) +: +- CometColumnarExchange (147) : +- CometHashAggregate (146) : +- CometProject (145) : +- CometBroadcastHashJoin (144) @@ -152,9 +152,9 @@ : : +- CometScan parquet spark_catalog.default.time_dim (137) : +- ReusedExchange (143) +- BroadcastExchange (171) - +- * HashAggregate (170) - +- Exchange (169) - +- * ColumnarToRow (168) + +- * ColumnarToRow (170) + +- CometHashAggregate (169) + +- CometColumnarExchange (168) +- CometHashAggregate (167) +- CometProject (166) +- CometBroadcastHashJoin (165) @@ -276,668 +276,652 @@ Input: [] Keys: [] Functions [1]: [partial_count(1)] -(23) ColumnarToRow [codegen id : 1] +(23) CometColumnarExchange Input [1]: [count#13] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(24) Exchange -Input [1]: [count#13] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] - -(25) HashAggregate [codegen id : 16] +(24) CometHashAggregate Input [1]: [count#13] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#14] -Results [1]: [count(1)#14 AS h8_30_to_9#15] + +(25) ColumnarToRow [codegen id : 8] +Input [1]: [h8_30_to_9#14] (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] +Output [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] 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 (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)) +Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] +Condition : ((isnotnull(ss_hdemo_sk#16) AND isnotnull(ss_sold_time_sk#15)) AND isnotnull(ss_store_sk#17)) (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] +Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] +Arguments: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17], [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17] (29) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#20] +Output [1]: [hd_demo_sk#19] (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 +Left output [3]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17] +Right output [1]: [hd_demo_sk#19] +Arguments: [ss_hdemo_sk#16], [hd_demo_sk#19], Inner, BuildRight (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] +Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, hd_demo_sk#19] +Arguments: [ss_sold_time_sk#15, ss_store_sk#17], [ss_sold_time_sk#15, ss_store_sk#17] (32) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#21, t_hour#22, t_minute#23] +Output [3]: [t_time_sk#20, t_hour#21, t_minute#22] 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 (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)) +Input [3]: [t_time_sk#20, t_hour#21, t_minute#22] +Condition : ((((isnotnull(t_hour#21) AND isnotnull(t_minute#22)) AND (t_hour#21 = 9)) AND (t_minute#22 < 30)) AND isnotnull(t_time_sk#20)) (34) CometProject -Input [3]: [t_time_sk#21, t_hour#22, t_minute#23] -Arguments: [t_time_sk#21], [t_time_sk#21] +Input [3]: [t_time_sk#20, t_hour#21, t_minute#22] +Arguments: [t_time_sk#20], [t_time_sk#20] (35) CometBroadcastExchange -Input [1]: [t_time_sk#21] -Arguments: [t_time_sk#21] +Input [1]: [t_time_sk#20] +Arguments: [t_time_sk#20] (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 +Left output [2]: [ss_sold_time_sk#15, ss_store_sk#17] +Right output [1]: [t_time_sk#20] +Arguments: [ss_sold_time_sk#15], [t_time_sk#20], Inner, BuildRight (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] +Input [3]: [ss_sold_time_sk#15, ss_store_sk#17, t_time_sk#20] +Arguments: [ss_store_sk#17], [ss_store_sk#17] (38) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#24] +Output [1]: [s_store_sk#23] (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 +Left output [1]: [ss_store_sk#17] +Right output [1]: [s_store_sk#23] +Arguments: [ss_store_sk#17], [s_store_sk#23], Inner, BuildRight (40) CometProject -Input [2]: [ss_store_sk#18, s_store_sk#24] +Input [2]: [ss_store_sk#17, s_store_sk#23] (41) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -(42) ColumnarToRow [codegen id : 2] -Input [1]: [count#25] +(42) CometColumnarExchange +Input [1]: [count#24] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(43) Exchange -Input [1]: [count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] - -(44) HashAggregate [codegen id : 3] -Input [1]: [count#25] +(43) CometHashAggregate +Input [1]: [count#24] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#26] -Results [1]: [count(1)#26 AS h9_to_9_30#27] + +(44) ColumnarToRow [codegen id : 1] +Input [1]: [h9_to_9_30#25] (45) BroadcastExchange -Input [1]: [h9_to_9_30#27] +Input [1]: [h9_to_9_30#25] Arguments: IdentityBroadcastMode, [plan_id=3] -(46) BroadcastNestedLoopJoin [codegen id : 16] +(46) BroadcastNestedLoopJoin [codegen id : 8] Join type: Inner Join condition: None (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] +Output [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] 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 (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)) +Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] +Condition : ((isnotnull(ss_hdemo_sk#27) AND isnotnull(ss_sold_time_sk#26)) AND isnotnull(ss_store_sk#28)) (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] +Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] +Arguments: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28], [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28] (50) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#32] +Output [1]: [hd_demo_sk#30] (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 +Left output [3]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28] +Right output [1]: [hd_demo_sk#30] +Arguments: [ss_hdemo_sk#27], [hd_demo_sk#30], Inner, BuildRight (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] +Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, hd_demo_sk#30] +Arguments: [ss_sold_time_sk#26, ss_store_sk#28], [ss_sold_time_sk#26, ss_store_sk#28] (53) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#33, t_hour#34, t_minute#35] +Output [3]: [t_time_sk#31, t_hour#32, t_minute#33] 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 (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)) +Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] +Condition : ((((isnotnull(t_hour#32) AND isnotnull(t_minute#33)) AND (t_hour#32 = 9)) AND (t_minute#33 >= 30)) AND isnotnull(t_time_sk#31)) (55) CometProject -Input [3]: [t_time_sk#33, t_hour#34, t_minute#35] -Arguments: [t_time_sk#33], [t_time_sk#33] +Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] +Arguments: [t_time_sk#31], [t_time_sk#31] (56) CometBroadcastExchange -Input [1]: [t_time_sk#33] -Arguments: [t_time_sk#33] +Input [1]: [t_time_sk#31] +Arguments: [t_time_sk#31] (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 +Left output [2]: [ss_sold_time_sk#26, ss_store_sk#28] +Right output [1]: [t_time_sk#31] +Arguments: [ss_sold_time_sk#26], [t_time_sk#31], Inner, BuildRight (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] +Input [3]: [ss_sold_time_sk#26, ss_store_sk#28, t_time_sk#31] +Arguments: [ss_store_sk#28], [ss_store_sk#28] (59) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#36] +Output [1]: [s_store_sk#34] (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 +Left output [1]: [ss_store_sk#28] +Right output [1]: [s_store_sk#34] +Arguments: [ss_store_sk#28], [s_store_sk#34], Inner, BuildRight (61) CometProject -Input [2]: [ss_store_sk#30, s_store_sk#36] +Input [2]: [ss_store_sk#28, s_store_sk#34] (62) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -(63) ColumnarToRow [codegen id : 4] -Input [1]: [count#37] +(63) CometColumnarExchange +Input [1]: [count#35] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(64) Exchange -Input [1]: [count#37] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] - -(65) HashAggregate [codegen id : 5] -Input [1]: [count#37] +(64) CometHashAggregate +Input [1]: [count#35] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#38] -Results [1]: [count(1)#38 AS h9_30_to_10#39] + +(65) ColumnarToRow [codegen id : 2] +Input [1]: [h9_30_to_10#36] (66) BroadcastExchange -Input [1]: [h9_30_to_10#39] +Input [1]: [h9_30_to_10#36] Arguments: IdentityBroadcastMode, [plan_id=5] -(67) BroadcastNestedLoopJoin [codegen id : 16] +(67) BroadcastNestedLoopJoin [codegen id : 8] Join type: Inner Join condition: None (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] +Output [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] 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 (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)) +Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] +Condition : ((isnotnull(ss_hdemo_sk#38) AND isnotnull(ss_sold_time_sk#37)) AND isnotnull(ss_store_sk#39)) (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] +Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] +Arguments: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39], [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39] (71) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#44] +Output [1]: [hd_demo_sk#41] (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 +Left output [3]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39] +Right output [1]: [hd_demo_sk#41] +Arguments: [ss_hdemo_sk#38], [hd_demo_sk#41], Inner, BuildRight (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] +Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, hd_demo_sk#41] +Arguments: [ss_sold_time_sk#37, ss_store_sk#39], [ss_sold_time_sk#37, ss_store_sk#39] (74) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#45, t_hour#46, t_minute#47] +Output [3]: [t_time_sk#42, t_hour#43, t_minute#44] 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 (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)) +Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] +Condition : ((((isnotnull(t_hour#43) AND isnotnull(t_minute#44)) AND (t_hour#43 = 10)) AND (t_minute#44 < 30)) AND isnotnull(t_time_sk#42)) (76) CometProject -Input [3]: [t_time_sk#45, t_hour#46, t_minute#47] -Arguments: [t_time_sk#45], [t_time_sk#45] +Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] +Arguments: [t_time_sk#42], [t_time_sk#42] (77) CometBroadcastExchange -Input [1]: [t_time_sk#45] -Arguments: [t_time_sk#45] +Input [1]: [t_time_sk#42] +Arguments: [t_time_sk#42] (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 +Left output [2]: [ss_sold_time_sk#37, ss_store_sk#39] +Right output [1]: [t_time_sk#42] +Arguments: [ss_sold_time_sk#37], [t_time_sk#42], Inner, BuildRight (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] +Input [3]: [ss_sold_time_sk#37, ss_store_sk#39, t_time_sk#42] +Arguments: [ss_store_sk#39], [ss_store_sk#39] (80) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#48] +Output [1]: [s_store_sk#45] (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 +Left output [1]: [ss_store_sk#39] +Right output [1]: [s_store_sk#45] +Arguments: [ss_store_sk#39], [s_store_sk#45], Inner, BuildRight (82) CometProject -Input [2]: [ss_store_sk#42, s_store_sk#48] +Input [2]: [ss_store_sk#39, s_store_sk#45] (83) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -(84) ColumnarToRow [codegen id : 6] -Input [1]: [count#49] +(84) CometColumnarExchange +Input [1]: [count#46] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(85) Exchange -Input [1]: [count#49] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] - -(86) HashAggregate [codegen id : 7] -Input [1]: [count#49] +(85) CometHashAggregate +Input [1]: [count#46] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#50] -Results [1]: [count(1)#50 AS h10_to_10_30#51] + +(86) ColumnarToRow [codegen id : 3] +Input [1]: [h10_to_10_30#47] (87) BroadcastExchange -Input [1]: [h10_to_10_30#51] +Input [1]: [h10_to_10_30#47] Arguments: IdentityBroadcastMode, [plan_id=7] -(88) BroadcastNestedLoopJoin [codegen id : 16] +(88) BroadcastNestedLoopJoin [codegen id : 8] Join type: Inner Join condition: None (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] +Output [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] 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 (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)) +Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] +Condition : ((isnotnull(ss_hdemo_sk#49) AND isnotnull(ss_sold_time_sk#48)) AND isnotnull(ss_store_sk#50)) (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] +Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] +Arguments: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50], [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50] (92) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#56] +Output [1]: [hd_demo_sk#52] (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 +Left output [3]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50] +Right output [1]: [hd_demo_sk#52] +Arguments: [ss_hdemo_sk#49], [hd_demo_sk#52], Inner, BuildRight (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] +Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, hd_demo_sk#52] +Arguments: [ss_sold_time_sk#48, ss_store_sk#50], [ss_sold_time_sk#48, ss_store_sk#50] (95) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#57, t_hour#58, t_minute#59] +Output [3]: [t_time_sk#53, t_hour#54, t_minute#55] 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 (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)) +Input [3]: [t_time_sk#53, t_hour#54, t_minute#55] +Condition : ((((isnotnull(t_hour#54) AND isnotnull(t_minute#55)) AND (t_hour#54 = 10)) AND (t_minute#55 >= 30)) AND isnotnull(t_time_sk#53)) (97) CometProject -Input [3]: [t_time_sk#57, t_hour#58, t_minute#59] -Arguments: [t_time_sk#57], [t_time_sk#57] +Input [3]: [t_time_sk#53, t_hour#54, t_minute#55] +Arguments: [t_time_sk#53], [t_time_sk#53] (98) CometBroadcastExchange -Input [1]: [t_time_sk#57] -Arguments: [t_time_sk#57] +Input [1]: [t_time_sk#53] +Arguments: [t_time_sk#53] (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 +Left output [2]: [ss_sold_time_sk#48, ss_store_sk#50] +Right output [1]: [t_time_sk#53] +Arguments: [ss_sold_time_sk#48], [t_time_sk#53], Inner, BuildRight (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] +Input [3]: [ss_sold_time_sk#48, ss_store_sk#50, t_time_sk#53] +Arguments: [ss_store_sk#50], [ss_store_sk#50] (101) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#60] +Output [1]: [s_store_sk#56] (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 +Left output [1]: [ss_store_sk#50] +Right output [1]: [s_store_sk#56] +Arguments: [ss_store_sk#50], [s_store_sk#56], Inner, BuildRight (103) CometProject -Input [2]: [ss_store_sk#54, s_store_sk#60] +Input [2]: [ss_store_sk#50, s_store_sk#56] (104) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -(105) ColumnarToRow [codegen id : 8] -Input [1]: [count#61] +(105) CometColumnarExchange +Input [1]: [count#57] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(106) Exchange -Input [1]: [count#61] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=8] - -(107) HashAggregate [codegen id : 9] -Input [1]: [count#61] +(106) CometHashAggregate +Input [1]: [count#57] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#62] -Results [1]: [count(1)#62 AS h10_30_to_11#63] + +(107) ColumnarToRow [codegen id : 4] +Input [1]: [h10_30_to_11#58] (108) BroadcastExchange -Input [1]: [h10_30_to_11#63] +Input [1]: [h10_30_to_11#58] Arguments: IdentityBroadcastMode, [plan_id=9] -(109) BroadcastNestedLoopJoin [codegen id : 16] +(109) BroadcastNestedLoopJoin [codegen id : 8] Join type: Inner Join condition: None (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] +Output [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] 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 (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)) +Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] +Condition : ((isnotnull(ss_hdemo_sk#60) AND isnotnull(ss_sold_time_sk#59)) AND isnotnull(ss_store_sk#61)) (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] +Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] +Arguments: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61], [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61] (113) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#68] +Output [1]: [hd_demo_sk#63] (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 +Left output [3]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61] +Right output [1]: [hd_demo_sk#63] +Arguments: [ss_hdemo_sk#60], [hd_demo_sk#63], Inner, BuildRight (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] +Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, hd_demo_sk#63] +Arguments: [ss_sold_time_sk#59, ss_store_sk#61], [ss_sold_time_sk#59, ss_store_sk#61] (116) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#69, t_hour#70, t_minute#71] +Output [3]: [t_time_sk#64, t_hour#65, t_minute#66] 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 (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)) +Input [3]: [t_time_sk#64, t_hour#65, t_minute#66] +Condition : ((((isnotnull(t_hour#65) AND isnotnull(t_minute#66)) AND (t_hour#65 = 11)) AND (t_minute#66 < 30)) AND isnotnull(t_time_sk#64)) (118) CometProject -Input [3]: [t_time_sk#69, t_hour#70, t_minute#71] -Arguments: [t_time_sk#69], [t_time_sk#69] +Input [3]: [t_time_sk#64, t_hour#65, t_minute#66] +Arguments: [t_time_sk#64], [t_time_sk#64] (119) CometBroadcastExchange -Input [1]: [t_time_sk#69] -Arguments: [t_time_sk#69] +Input [1]: [t_time_sk#64] +Arguments: [t_time_sk#64] (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 +Left output [2]: [ss_sold_time_sk#59, ss_store_sk#61] +Right output [1]: [t_time_sk#64] +Arguments: [ss_sold_time_sk#59], [t_time_sk#64], Inner, BuildRight (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] +Input [3]: [ss_sold_time_sk#59, ss_store_sk#61, t_time_sk#64] +Arguments: [ss_store_sk#61], [ss_store_sk#61] (122) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#72] +Output [1]: [s_store_sk#67] (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 +Left output [1]: [ss_store_sk#61] +Right output [1]: [s_store_sk#67] +Arguments: [ss_store_sk#61], [s_store_sk#67], Inner, BuildRight (124) CometProject -Input [2]: [ss_store_sk#66, s_store_sk#72] +Input [2]: [ss_store_sk#61, s_store_sk#67] (125) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -(126) ColumnarToRow [codegen id : 10] -Input [1]: [count#73] +(126) CometColumnarExchange +Input [1]: [count#68] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(127) Exchange -Input [1]: [count#73] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] - -(128) HashAggregate [codegen id : 11] -Input [1]: [count#73] +(127) CometHashAggregate +Input [1]: [count#68] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#74] -Results [1]: [count(1)#74 AS h11_to_11_30#75] + +(128) ColumnarToRow [codegen id : 5] +Input [1]: [h11_to_11_30#69] (129) BroadcastExchange -Input [1]: [h11_to_11_30#75] +Input [1]: [h11_to_11_30#69] Arguments: IdentityBroadcastMode, [plan_id=11] -(130) BroadcastNestedLoopJoin [codegen id : 16] +(130) BroadcastNestedLoopJoin [codegen id : 8] Join type: Inner Join condition: None (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] +Output [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] 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 (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)) +Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] +Condition : ((isnotnull(ss_hdemo_sk#71) AND isnotnull(ss_sold_time_sk#70)) AND isnotnull(ss_store_sk#72)) (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] +Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] +Arguments: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72], [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72] (134) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#80] +Output [1]: [hd_demo_sk#74] (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 +Left output [3]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72] +Right output [1]: [hd_demo_sk#74] +Arguments: [ss_hdemo_sk#71], [hd_demo_sk#74], Inner, BuildRight (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] +Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, hd_demo_sk#74] +Arguments: [ss_sold_time_sk#70, ss_store_sk#72], [ss_sold_time_sk#70, ss_store_sk#72] (137) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#81, t_hour#82, t_minute#83] +Output [3]: [t_time_sk#75, t_hour#76, t_minute#77] 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 (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)) +Input [3]: [t_time_sk#75, t_hour#76, t_minute#77] +Condition : ((((isnotnull(t_hour#76) AND isnotnull(t_minute#77)) AND (t_hour#76 = 11)) AND (t_minute#77 >= 30)) AND isnotnull(t_time_sk#75)) (139) CometProject -Input [3]: [t_time_sk#81, t_hour#82, t_minute#83] -Arguments: [t_time_sk#81], [t_time_sk#81] +Input [3]: [t_time_sk#75, t_hour#76, t_minute#77] +Arguments: [t_time_sk#75], [t_time_sk#75] (140) CometBroadcastExchange -Input [1]: [t_time_sk#81] -Arguments: [t_time_sk#81] +Input [1]: [t_time_sk#75] +Arguments: [t_time_sk#75] (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 +Left output [2]: [ss_sold_time_sk#70, ss_store_sk#72] +Right output [1]: [t_time_sk#75] +Arguments: [ss_sold_time_sk#70], [t_time_sk#75], Inner, BuildRight (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] +Input [3]: [ss_sold_time_sk#70, ss_store_sk#72, t_time_sk#75] +Arguments: [ss_store_sk#72], [ss_store_sk#72] (143) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#84] +Output [1]: [s_store_sk#78] (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 +Left output [1]: [ss_store_sk#72] +Right output [1]: [s_store_sk#78] +Arguments: [ss_store_sk#72], [s_store_sk#78], Inner, BuildRight (145) CometProject -Input [2]: [ss_store_sk#78, s_store_sk#84] +Input [2]: [ss_store_sk#72, s_store_sk#78] (146) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -(147) ColumnarToRow [codegen id : 12] -Input [1]: [count#85] +(147) CometColumnarExchange +Input [1]: [count#79] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(148) Exchange -Input [1]: [count#85] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] - -(149) HashAggregate [codegen id : 13] -Input [1]: [count#85] +(148) CometHashAggregate +Input [1]: [count#79] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#86] -Results [1]: [count(1)#86 AS h11_30_to_12#87] + +(149) ColumnarToRow [codegen id : 6] +Input [1]: [h11_30_to_12#80] (150) BroadcastExchange -Input [1]: [h11_30_to_12#87] +Input [1]: [h11_30_to_12#80] Arguments: IdentityBroadcastMode, [plan_id=13] -(151) BroadcastNestedLoopJoin [codegen id : 16] +(151) BroadcastNestedLoopJoin [codegen id : 8] Join type: Inner Join condition: None (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] +Output [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] 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 (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)) +Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] +Condition : ((isnotnull(ss_hdemo_sk#82) AND isnotnull(ss_sold_time_sk#81)) AND isnotnull(ss_store_sk#83)) (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] +Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] +Arguments: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83], [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83] (155) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#92] +Output [1]: [hd_demo_sk#85] (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 +Left output [3]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83] +Right output [1]: [hd_demo_sk#85] +Arguments: [ss_hdemo_sk#82], [hd_demo_sk#85], Inner, BuildRight (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] +Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, hd_demo_sk#85] +Arguments: [ss_sold_time_sk#81, ss_store_sk#83], [ss_sold_time_sk#81, ss_store_sk#83] (158) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#93, t_hour#94, t_minute#95] +Output [3]: [t_time_sk#86, t_hour#87, t_minute#88] 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 (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)) +Input [3]: [t_time_sk#86, t_hour#87, t_minute#88] +Condition : ((((isnotnull(t_hour#87) AND isnotnull(t_minute#88)) AND (t_hour#87 = 12)) AND (t_minute#88 < 30)) AND isnotnull(t_time_sk#86)) (160) CometProject -Input [3]: [t_time_sk#93, t_hour#94, t_minute#95] -Arguments: [t_time_sk#93], [t_time_sk#93] +Input [3]: [t_time_sk#86, t_hour#87, t_minute#88] +Arguments: [t_time_sk#86], [t_time_sk#86] (161) CometBroadcastExchange -Input [1]: [t_time_sk#93] -Arguments: [t_time_sk#93] +Input [1]: [t_time_sk#86] +Arguments: [t_time_sk#86] (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 +Left output [2]: [ss_sold_time_sk#81, ss_store_sk#83] +Right output [1]: [t_time_sk#86] +Arguments: [ss_sold_time_sk#81], [t_time_sk#86], Inner, BuildRight (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] +Input [3]: [ss_sold_time_sk#81, ss_store_sk#83, t_time_sk#86] +Arguments: [ss_store_sk#83], [ss_store_sk#83] (164) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#96] +Output [1]: [s_store_sk#89] (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 +Left output [1]: [ss_store_sk#83] +Right output [1]: [s_store_sk#89] +Arguments: [ss_store_sk#83], [s_store_sk#89], Inner, BuildRight (166) CometProject -Input [2]: [ss_store_sk#90, s_store_sk#96] +Input [2]: [ss_store_sk#83, s_store_sk#89] (167) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -(168) ColumnarToRow [codegen id : 14] -Input [1]: [count#97] +(168) CometColumnarExchange +Input [1]: [count#90] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] -(169) Exchange -Input [1]: [count#97] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=14] - -(170) HashAggregate [codegen id : 15] -Input [1]: [count#97] +(169) CometHashAggregate +Input [1]: [count#90] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#98] -Results [1]: [count(1)#98 AS h12_to_12_30#99] + +(170) ColumnarToRow [codegen id : 7] +Input [1]: [h12_to_12_30#91] (171) BroadcastExchange -Input [1]: [h12_to_12_30#99] +Input [1]: [h12_to_12_30#91] Arguments: IdentityBroadcastMode, [plan_id=15] -(172) BroadcastNestedLoopJoin [codegen id : 16] +(172) BroadcastNestedLoopJoin [codegen id : 8] 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 d3af10d19..b01685ed0 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 (16) +WholeStageCodegen (8) BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin @@ -6,93 +6,39 @@ WholeStageCodegen (16) BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin - HashAggregate [count] [count(1),h8_30_to_9,count] + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] - CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_store_name] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] + CometHashAggregate [h8_30_to_9,count,count(1)] + CometColumnarExchange #1 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometBroadcastExchange [hd_demo_sk] #2 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange [t_time_sk] #3 + CometProject [t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + CometBroadcastExchange [s_store_sk] #4 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_store_name] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - HashAggregate [count] [count(1),h9_to_9_30,count] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange #6 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] - CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #7 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - HashAggregate [count] [count(1),h9_30_to_10,count] - InputAdapter - Exchange #9 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] - CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #10 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (7) - HashAggregate [count] [count(1),h10_to_10_30,count] - InputAdapter - Exchange #12 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter + CometHashAggregate [h9_to_9_30,count,count(1)] + CometColumnarExchange #6 CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] @@ -104,20 +50,18 @@ WholeStageCodegen (16) CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #13 + CometBroadcastExchange [t_time_sk] #7 CometProject [t_time_sk] CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (9) - HashAggregate [count] [count(1),h10_30_to_11,count] - InputAdapter - Exchange #15 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometHashAggregate [h9_30_to_10,count,count(1)] + CometColumnarExchange #9 CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] @@ -129,20 +73,18 @@ WholeStageCodegen (16) CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #16 + CometBroadcastExchange [t_time_sk] #10 CometProject [t_time_sk] CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (11) - HashAggregate [count] [count(1),h11_to_11_30,count] - InputAdapter - Exchange #18 - WholeStageCodegen (10) - ColumnarToRow - InputAdapter + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometHashAggregate [h10_to_10_30,count,count(1)] + CometColumnarExchange #12 CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] @@ -154,20 +96,18 @@ WholeStageCodegen (16) CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #19 + CometBroadcastExchange [t_time_sk] #13 CometProject [t_time_sk] CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #20 - WholeStageCodegen (13) - HashAggregate [count] [count(1),h11_30_to_12,count] - InputAdapter - Exchange #21 - WholeStageCodegen (12) - ColumnarToRow - InputAdapter + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometHashAggregate [h10_30_to_11,count,count(1)] + CometColumnarExchange #15 CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] @@ -179,20 +119,18 @@ WholeStageCodegen (16) CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #22 + CometBroadcastExchange [t_time_sk] #16 CometProject [t_time_sk] CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #23 - WholeStageCodegen (15) - HashAggregate [count] [count(1),h12_to_12_30,count] - InputAdapter - Exchange #24 - WholeStageCodegen (14) - ColumnarToRow - InputAdapter + InputAdapter + BroadcastExchange #17 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometHashAggregate [h11_to_11_30,count,count(1)] + CometColumnarExchange #18 CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] @@ -204,8 +142,54 @@ WholeStageCodegen (16) CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #25 + CometBroadcastExchange [t_time_sk] #19 CometProject [t_time_sk] CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 + InputAdapter + BroadcastExchange #20 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometHashAggregate [h11_30_to_12,count,count(1)] + CometColumnarExchange #21 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange [t_time_sk] #22 + CometProject [t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + ReusedExchange [s_store_sk] #4 + InputAdapter + BroadcastExchange #23 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometHashAggregate [h12_to_12_30,count,count(1)] + CometColumnarExchange #24 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange [t_time_sk] #25 + CometProject [t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] + 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 07761b14b..e4dca6fb4 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,32 +1,36 @@ == Physical Plan == -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) +TakeOrderedAndProject (32) ++- * Project (31) + +- * Filter (30) + +- Window (29) + +- * ColumnarToRow (28) + +- CometSort (27) + +- CometColumnarExchange (26) + +- RowToColumnar (25) + +- * HashAggregate (24) + +- * ColumnarToRow (23) + +- CometColumnarExchange (22) + +- RowToColumnar (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 @@ -127,70 +131,82 @@ Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#7))] 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] -(21) Exchange +(21) RowToColumnar 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] -(22) HashAggregate [codegen id : 2] +(22) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(23) ColumnarToRow [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] + +(24) 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))#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] -(23) Exchange +(25) RowToColumnar +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] + +(26) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] + +(27) CometSort 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] +Arguments: [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], [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] -(24) Sort [codegen id : 3] +(28) ColumnarToRow [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 -(25) Window +(29) 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] -(26) Filter [codegen id : 4] +(30) 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 -(27) Project [codegen id : 4] +(31) 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] -(28) TakeOrderedAndProject +(32) 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 = 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) +BroadcastExchange (37) ++- * ColumnarToRow (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) -(29) Scan parquet spark_catalog.default.date_dim +(33) 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 -(30) CometFilter +(34) 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)) -(31) CometProject +(35) 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] -(32) ColumnarToRow [codegen id : 1] +(36) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_moy#12] -(33) BroadcastExchange +(37) 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 2bfff90d3..ec9252b0f 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 @@ -5,40 +5,44 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla InputAdapter Window [_w0,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (3) - Sort [i_category,i_brand,s_store_name,s_company_name] + ColumnarToRow InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name] #1 - 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 (1) - HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] - CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometFilter [i_item_sk,i_brand,i_class,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #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] + CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] + CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_class,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_moy] #5 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/explain.txt index 69c23211b..b4a81109d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/explain.txt @@ -26,14 +26,16 @@ Input [1]: [r_reason_sk#1] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#2, [id=#3] -* Project (12) -+- * HashAggregate (11) - +- Exchange (10) - +- * HashAggregate (9) - +- * ColumnarToRow (8) - +- CometProject (7) - +- CometFilter (6) - +- CometScan parquet spark_catalog.default.store_sales (5) +* Project (14) ++- * HashAggregate (13) + +- * ColumnarToRow (12) + +- CometColumnarExchange (11) + +- RowToColumnar (10) + +- * HashAggregate (9) + +- * ColumnarToRow (8) + +- CometProject (7) + +- CometFilter (6) + +- CometScan parquet spark_catalog.default.store_sales (5) (5) Scan parquet spark_catalog.default.store_sales @@ -61,18 +63,24 @@ Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt# Aggregate Attributes [5]: [count#21, sum#22, count#23, sum#24, count#25] Results [5]: [count#26, sum#27, count#28, sum#29, count#30] -(10) Exchange +(10) RowToColumnar Input [5]: [count#26, sum#27, count#28, sum#29, count#30] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] -(11) HashAggregate [codegen id : 2] +(11) CometColumnarExchange +Input [5]: [count#26, sum#27, count#28, sum#29, count#30] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(12) ColumnarToRow [codegen id : 2] +Input [5]: [count#26, sum#27, count#28, sum#29, count#30] + +(13) HashAggregate [codegen id : 2] Input [5]: [count#26, sum#27, count#28, sum#29, count#30] Keys: [] Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#18)), avg(UnscaledValue(ss_net_paid#19))] Aggregate Attributes [3]: [count(1)#31, avg(UnscaledValue(ss_ext_discount_amt#18))#32, avg(UnscaledValue(ss_net_paid#19))#33] Results [3]: [count(1)#31 AS count(1)#34, cast((avg(UnscaledValue(ss_ext_discount_amt#18))#32 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#35, cast((avg(UnscaledValue(ss_net_paid#19))#33 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#36] -(12) Project [codegen id : 2] +(14) Project [codegen id : 2] Output [1]: [named_struct(count(1), count(1)#34, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#35, avg(ss_net_paid), avg(ss_net_paid)#36) AS mergedValue#37] Input [3]: [count(1)#34, avg(ss_ext_discount_amt)#35, avg(ss_net_paid)#36] @@ -81,53 +89,61 @@ Subquery:2 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery Subquery:3 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#5, [id=#6] -* Project (20) -+- * HashAggregate (19) - +- Exchange (18) - +- * HashAggregate (17) - +- * ColumnarToRow (16) - +- CometProject (15) - +- CometFilter (14) - +- CometScan parquet spark_catalog.default.store_sales (13) - - -(13) Scan parquet spark_catalog.default.store_sales +* Project (24) ++- * HashAggregate (23) + +- * ColumnarToRow (22) + +- CometColumnarExchange (21) + +- RowToColumnar (20) + +- * HashAggregate (19) + +- * ColumnarToRow (18) + +- CometProject (17) + +- CometFilter (16) + +- CometScan parquet spark_catalog.default.store_sales (15) + + +(15) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_quantity#38, ss_ext_discount_amt#39, ss_net_paid#40, ss_sold_date_sk#41] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct -(14) CometFilter +(16) CometFilter Input [4]: [ss_quantity#38, ss_ext_discount_amt#39, ss_net_paid#40, ss_sold_date_sk#41] Condition : ((isnotnull(ss_quantity#38) AND (ss_quantity#38 >= 21)) AND (ss_quantity#38 <= 40)) -(15) CometProject +(17) CometProject Input [4]: [ss_quantity#38, ss_ext_discount_amt#39, ss_net_paid#40, ss_sold_date_sk#41] Arguments: [ss_ext_discount_amt#39, ss_net_paid#40], [ss_ext_discount_amt#39, ss_net_paid#40] -(16) ColumnarToRow [codegen id : 1] +(18) ColumnarToRow [codegen id : 1] Input [2]: [ss_ext_discount_amt#39, ss_net_paid#40] -(17) HashAggregate [codegen id : 1] +(19) HashAggregate [codegen id : 1] Input [2]: [ss_ext_discount_amt#39, ss_net_paid#40] Keys: [] Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#39)), partial_avg(UnscaledValue(ss_net_paid#40))] Aggregate Attributes [5]: [count#42, sum#43, count#44, sum#45, count#46] Results [5]: [count#47, sum#48, count#49, sum#50, count#51] -(18) Exchange +(20) RowToColumnar +Input [5]: [count#47, sum#48, count#49, sum#50, count#51] + +(21) CometColumnarExchange Input [5]: [count#47, sum#48, count#49, sum#50, count#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(19) HashAggregate [codegen id : 2] +(22) ColumnarToRow [codegen id : 2] +Input [5]: [count#47, sum#48, count#49, sum#50, count#51] + +(23) HashAggregate [codegen id : 2] Input [5]: [count#47, sum#48, count#49, sum#50, count#51] Keys: [] Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#39)), avg(UnscaledValue(ss_net_paid#40))] Aggregate Attributes [3]: [count(1)#52, avg(UnscaledValue(ss_ext_discount_amt#39))#53, avg(UnscaledValue(ss_net_paid#40))#54] Results [3]: [count(1)#52 AS count(1)#55, cast((avg(UnscaledValue(ss_ext_discount_amt#39))#53 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#56, cast((avg(UnscaledValue(ss_net_paid#40))#54 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#57] -(20) Project [codegen id : 2] +(24) Project [codegen id : 2] Output [1]: [named_struct(count(1), count(1)#55, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#56, avg(ss_net_paid), avg(ss_net_paid)#57) AS mergedValue#58] Input [3]: [count(1)#55, avg(ss_ext_discount_amt)#56, avg(ss_net_paid)#57] @@ -136,53 +152,61 @@ Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery Subquery:6 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#8, [id=#9] -* Project (28) -+- * HashAggregate (27) - +- Exchange (26) - +- * HashAggregate (25) - +- * ColumnarToRow (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan parquet spark_catalog.default.store_sales (21) - - -(21) Scan parquet spark_catalog.default.store_sales +* Project (34) ++- * HashAggregate (33) + +- * ColumnarToRow (32) + +- CometColumnarExchange (31) + +- RowToColumnar (30) + +- * HashAggregate (29) + +- * ColumnarToRow (28) + +- CometProject (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.store_sales (25) + + +(25) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_quantity#59, ss_ext_discount_amt#60, ss_net_paid#61, ss_sold_date_sk#62] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct -(22) CometFilter +(26) CometFilter Input [4]: [ss_quantity#59, ss_ext_discount_amt#60, ss_net_paid#61, ss_sold_date_sk#62] Condition : ((isnotnull(ss_quantity#59) AND (ss_quantity#59 >= 41)) AND (ss_quantity#59 <= 60)) -(23) CometProject +(27) CometProject Input [4]: [ss_quantity#59, ss_ext_discount_amt#60, ss_net_paid#61, ss_sold_date_sk#62] Arguments: [ss_ext_discount_amt#60, ss_net_paid#61], [ss_ext_discount_amt#60, ss_net_paid#61] -(24) ColumnarToRow [codegen id : 1] +(28) ColumnarToRow [codegen id : 1] Input [2]: [ss_ext_discount_amt#60, ss_net_paid#61] -(25) HashAggregate [codegen id : 1] +(29) HashAggregate [codegen id : 1] Input [2]: [ss_ext_discount_amt#60, ss_net_paid#61] Keys: [] Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#60)), partial_avg(UnscaledValue(ss_net_paid#61))] Aggregate Attributes [5]: [count#63, sum#64, count#65, sum#66, count#67] Results [5]: [count#68, sum#69, count#70, sum#71, count#72] -(26) Exchange +(30) RowToColumnar +Input [5]: [count#68, sum#69, count#70, sum#71, count#72] + +(31) CometColumnarExchange +Input [5]: [count#68, sum#69, count#70, sum#71, count#72] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(32) ColumnarToRow [codegen id : 2] Input [5]: [count#68, sum#69, count#70, sum#71, count#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] -(27) HashAggregate [codegen id : 2] +(33) HashAggregate [codegen id : 2] Input [5]: [count#68, sum#69, count#70, sum#71, count#72] Keys: [] Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#60)), avg(UnscaledValue(ss_net_paid#61))] Aggregate Attributes [3]: [count(1)#73, avg(UnscaledValue(ss_ext_discount_amt#60))#74, avg(UnscaledValue(ss_net_paid#61))#75] Results [3]: [count(1)#73 AS count(1)#76, cast((avg(UnscaledValue(ss_ext_discount_amt#60))#74 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#77, cast((avg(UnscaledValue(ss_net_paid#61))#75 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#78] -(28) Project [codegen id : 2] +(34) Project [codegen id : 2] Output [1]: [named_struct(count(1), count(1)#76, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#77, avg(ss_net_paid), avg(ss_net_paid)#78) AS mergedValue#79] Input [3]: [count(1)#76, avg(ss_ext_discount_amt)#77, avg(ss_net_paid)#78] @@ -191,53 +215,61 @@ Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery Subquery:9 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* Project (36) -+- * HashAggregate (35) - +- Exchange (34) - +- * HashAggregate (33) - +- * ColumnarToRow (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan parquet spark_catalog.default.store_sales (29) +* Project (44) ++- * HashAggregate (43) + +- * ColumnarToRow (42) + +- CometColumnarExchange (41) + +- RowToColumnar (40) + +- * HashAggregate (39) + +- * ColumnarToRow (38) + +- CometProject (37) + +- CometFilter (36) + +- CometScan parquet spark_catalog.default.store_sales (35) -(29) Scan parquet spark_catalog.default.store_sales +(35) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_quantity#80, ss_ext_discount_amt#81, ss_net_paid#82, ss_sold_date_sk#83] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct -(30) CometFilter +(36) CometFilter Input [4]: [ss_quantity#80, ss_ext_discount_amt#81, ss_net_paid#82, ss_sold_date_sk#83] Condition : ((isnotnull(ss_quantity#80) AND (ss_quantity#80 >= 61)) AND (ss_quantity#80 <= 80)) -(31) CometProject +(37) CometProject Input [4]: [ss_quantity#80, ss_ext_discount_amt#81, ss_net_paid#82, ss_sold_date_sk#83] Arguments: [ss_ext_discount_amt#81, ss_net_paid#82], [ss_ext_discount_amt#81, ss_net_paid#82] -(32) ColumnarToRow [codegen id : 1] +(38) ColumnarToRow [codegen id : 1] Input [2]: [ss_ext_discount_amt#81, ss_net_paid#82] -(33) HashAggregate [codegen id : 1] +(39) HashAggregate [codegen id : 1] Input [2]: [ss_ext_discount_amt#81, ss_net_paid#82] Keys: [] Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#81)), partial_avg(UnscaledValue(ss_net_paid#82))] Aggregate Attributes [5]: [count#84, sum#85, count#86, sum#87, count#88] Results [5]: [count#89, sum#90, count#91, sum#92, count#93] -(34) Exchange +(40) RowToColumnar +Input [5]: [count#89, sum#90, count#91, sum#92, count#93] + +(41) CometColumnarExchange Input [5]: [count#89, sum#90, count#91, sum#92, count#93] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(35) HashAggregate [codegen id : 2] +(42) ColumnarToRow [codegen id : 2] +Input [5]: [count#89, sum#90, count#91, sum#92, count#93] + +(43) HashAggregate [codegen id : 2] Input [5]: [count#89, sum#90, count#91, sum#92, count#93] Keys: [] Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#81)), avg(UnscaledValue(ss_net_paid#82))] Aggregate Attributes [3]: [count(1)#94, avg(UnscaledValue(ss_ext_discount_amt#81))#95, avg(UnscaledValue(ss_net_paid#82))#96] Results [3]: [count(1)#94 AS count(1)#97, cast((avg(UnscaledValue(ss_ext_discount_amt#81))#95 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#98, cast((avg(UnscaledValue(ss_net_paid#82))#96 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#99] -(36) Project [codegen id : 2] +(44) Project [codegen id : 2] Output [1]: [named_struct(count(1), count(1)#97, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#98, avg(ss_net_paid), avg(ss_net_paid)#99) AS mergedValue#100] Input [3]: [count(1)#97, avg(ss_ext_discount_amt)#98, avg(ss_net_paid)#99] @@ -246,53 +278,61 @@ Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery Subquery:12 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#14, [id=#15] -* Project (44) -+- * HashAggregate (43) - +- Exchange (42) - +- * HashAggregate (41) - +- * ColumnarToRow (40) - +- CometProject (39) - +- CometFilter (38) - +- CometScan parquet spark_catalog.default.store_sales (37) - - -(37) Scan parquet spark_catalog.default.store_sales +* Project (54) ++- * HashAggregate (53) + +- * ColumnarToRow (52) + +- CometColumnarExchange (51) + +- RowToColumnar (50) + +- * HashAggregate (49) + +- * ColumnarToRow (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan parquet spark_catalog.default.store_sales (45) + + +(45) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_quantity#101, ss_ext_discount_amt#102, ss_net_paid#103, ss_sold_date_sk#104] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct -(38) CometFilter +(46) CometFilter Input [4]: [ss_quantity#101, ss_ext_discount_amt#102, ss_net_paid#103, ss_sold_date_sk#104] Condition : ((isnotnull(ss_quantity#101) AND (ss_quantity#101 >= 81)) AND (ss_quantity#101 <= 100)) -(39) CometProject +(47) CometProject Input [4]: [ss_quantity#101, ss_ext_discount_amt#102, ss_net_paid#103, ss_sold_date_sk#104] Arguments: [ss_ext_discount_amt#102, ss_net_paid#103], [ss_ext_discount_amt#102, ss_net_paid#103] -(40) ColumnarToRow [codegen id : 1] +(48) ColumnarToRow [codegen id : 1] Input [2]: [ss_ext_discount_amt#102, ss_net_paid#103] -(41) HashAggregate [codegen id : 1] +(49) HashAggregate [codegen id : 1] Input [2]: [ss_ext_discount_amt#102, ss_net_paid#103] Keys: [] Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#102)), partial_avg(UnscaledValue(ss_net_paid#103))] Aggregate Attributes [5]: [count#105, sum#106, count#107, sum#108, count#109] Results [5]: [count#110, sum#111, count#112, sum#113, count#114] -(42) Exchange +(50) RowToColumnar Input [5]: [count#110, sum#111, count#112, sum#113, count#114] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5] -(43) HashAggregate [codegen id : 2] +(51) CometColumnarExchange +Input [5]: [count#110, sum#111, count#112, sum#113, count#114] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(52) ColumnarToRow [codegen id : 2] +Input [5]: [count#110, sum#111, count#112, sum#113, count#114] + +(53) HashAggregate [codegen id : 2] Input [5]: [count#110, sum#111, count#112, sum#113, count#114] Keys: [] Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#102)), avg(UnscaledValue(ss_net_paid#103))] Aggregate Attributes [3]: [count(1)#115, avg(UnscaledValue(ss_ext_discount_amt#102))#116, avg(UnscaledValue(ss_net_paid#103))#117] Results [3]: [count(1)#115 AS count(1)#118, cast((avg(UnscaledValue(ss_ext_discount_amt#102))#116 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#119, cast((avg(UnscaledValue(ss_net_paid#103))#117 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#120] -(44) Project [codegen id : 2] +(54) Project [codegen id : 2] Output [1]: [named_struct(count(1), count(1)#118, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#119, avg(ss_net_paid), avg(ss_net_paid)#120) AS mergedValue#121] Input [3]: [count(1)#118, avg(ss_ext_discount_amt)#119, avg(ss_net_paid)#120] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/simplified.txt index 581739cc6..79896e05b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/simplified.txt @@ -4,75 +4,85 @@ WholeStageCodegen (1) WholeStageCodegen (2) Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] - InputAdapter - Exchange #1 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] - ColumnarToRow - InputAdapter - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + ColumnarToRow + InputAdapter + CometColumnarExchange #1 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + ColumnarToRow + InputAdapter + CometProject [ss_ext_discount_amt,ss_net_paid] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #1 ReusedSubquery [mergedValue] #1 Subquery #2 WholeStageCodegen (2) Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] - InputAdapter - Exchange #2 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] - ColumnarToRow - InputAdapter - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + ColumnarToRow + InputAdapter + CometColumnarExchange #2 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + ColumnarToRow + InputAdapter + CometProject [ss_ext_discount_amt,ss_net_paid] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #2 ReusedSubquery [mergedValue] #2 Subquery #3 WholeStageCodegen (2) Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] - InputAdapter - Exchange #3 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] - ColumnarToRow - InputAdapter - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + ColumnarToRow + InputAdapter + CometColumnarExchange #3 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + ColumnarToRow + InputAdapter + CometProject [ss_ext_discount_amt,ss_net_paid] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #3 ReusedSubquery [mergedValue] #3 Subquery #4 WholeStageCodegen (2) Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] - InputAdapter - Exchange #4 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] - ColumnarToRow - InputAdapter - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + ColumnarToRow + InputAdapter + CometColumnarExchange #4 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + ColumnarToRow + InputAdapter + CometProject [ss_ext_discount_amt,ss_net_paid] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #4 ReusedSubquery [mergedValue] #4 Subquery #5 WholeStageCodegen (2) Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] - InputAdapter - Exchange #5 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] - ColumnarToRow - InputAdapter - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + ColumnarToRow + InputAdapter + CometColumnarExchange #5 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + ColumnarToRow + InputAdapter + CometProject [ss_ext_discount_amt,ss_net_paid] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #5 ReusedSubquery [mergedValue] #5 ColumnarToRow 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 dbea5e75d..ab17ec1d0 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,9 +1,9 @@ == Physical Plan == * Project (47) +- * BroadcastNestedLoopJoin Inner BuildRight (46) - :- * HashAggregate (25) - : +- Exchange (24) - : +- * ColumnarToRow (23) + :- * ColumnarToRow (25) + : +- CometHashAggregate (24) + : +- CometColumnarExchange (23) : +- CometHashAggregate (22) : +- CometProject (21) : +- CometBroadcastHashJoin (20) @@ -27,9 +27,9 @@ : +- CometFilter (17) : +- CometScan parquet spark_catalog.default.web_page (16) +- BroadcastExchange (45) - +- * HashAggregate (44) - +- Exchange (43) - +- * ColumnarToRow (42) + +- * ColumnarToRow (44) + +- CometHashAggregate (43) + +- CometColumnarExchange (42) +- CometHashAggregate (41) +- CometProject (40) +- CometBroadcastHashJoin (39) @@ -151,114 +151,110 @@ Input: [] Keys: [] Functions [1]: [partial_count(1)] -(23) ColumnarToRow [codegen id : 1] +(23) CometColumnarExchange Input [1]: [count#11] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(24) Exchange -Input [1]: [count#11] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] - -(25) HashAggregate [codegen id : 4] +(24) CometHashAggregate Input [1]: [count#11] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#12] -Results [1]: [count(1)#12 AS amc#13] + +(25) ColumnarToRow [codegen id : 2] +Input [1]: [amc#12] (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] +Output [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] 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 (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)) +Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] +Condition : ((isnotnull(ws_ship_hdemo_sk#14) AND isnotnull(ws_sold_time_sk#13)) AND isnotnull(ws_web_page_sk#15)) (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] +Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] +Arguments: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15], [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15] (29) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#18] +Output [1]: [hd_demo_sk#17] (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 +Left output [3]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15] +Right output [1]: [hd_demo_sk#17] +Arguments: [ws_ship_hdemo_sk#14], [hd_demo_sk#17], Inner, BuildRight (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] +Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, hd_demo_sk#17] +Arguments: [ws_sold_time_sk#13, ws_web_page_sk#15], [ws_sold_time_sk#13, ws_web_page_sk#15] (32) Scan parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#19, t_hour#20] +Output [2]: [t_time_sk#18, t_hour#19] 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 (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)) +Input [2]: [t_time_sk#18, t_hour#19] +Condition : (((isnotnull(t_hour#19) AND (t_hour#19 >= 19)) AND (t_hour#19 <= 20)) AND isnotnull(t_time_sk#18)) (34) CometProject -Input [2]: [t_time_sk#19, t_hour#20] -Arguments: [t_time_sk#19], [t_time_sk#19] +Input [2]: [t_time_sk#18, t_hour#19] +Arguments: [t_time_sk#18], [t_time_sk#18] (35) CometBroadcastExchange -Input [1]: [t_time_sk#19] -Arguments: [t_time_sk#19] +Input [1]: [t_time_sk#18] +Arguments: [t_time_sk#18] (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 +Left output [2]: [ws_sold_time_sk#13, ws_web_page_sk#15] +Right output [1]: [t_time_sk#18] +Arguments: [ws_sold_time_sk#13], [t_time_sk#18], Inner, BuildRight (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] +Input [3]: [ws_sold_time_sk#13, ws_web_page_sk#15, t_time_sk#18] +Arguments: [ws_web_page_sk#15], [ws_web_page_sk#15] (38) ReusedExchange [Reuses operator id: 19] -Output [1]: [wp_web_page_sk#21] +Output [1]: [wp_web_page_sk#20] (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 +Left output [1]: [ws_web_page_sk#15] +Right output [1]: [wp_web_page_sk#20] +Arguments: [ws_web_page_sk#15], [wp_web_page_sk#20], Inner, BuildRight (40) CometProject -Input [2]: [ws_web_page_sk#16, wp_web_page_sk#21] +Input [2]: [ws_web_page_sk#15, wp_web_page_sk#20] (41) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -(42) ColumnarToRow [codegen id : 2] -Input [1]: [count#22] +(42) CometColumnarExchange +Input [1]: [count#21] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(43) Exchange -Input [1]: [count#22] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] - -(44) HashAggregate [codegen id : 3] -Input [1]: [count#22] +(43) CometHashAggregate +Input [1]: [count#21] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#23] -Results [1]: [count(1)#23 AS pmc#24] + +(44) ColumnarToRow [codegen id : 1] +Input [1]: [pmc#22] (45) BroadcastExchange -Input [1]: [pmc#24] +Input [1]: [pmc#22] Arguments: IdentityBroadcastMode, [plan_id=3] -(46) BroadcastNestedLoopJoin [codegen id : 4] +(46) BroadcastNestedLoopJoin [codegen id : 2] Join type: Inner Join condition: None -(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] +(47) Project [codegen id : 2] +Output [1]: [(cast(amc#12 as decimal(15,4)) / cast(pmc#22 as decimal(15,4))) AS am_pm_ratio#23] +Input [2]: [amc#12, pmc#22] 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 44159cc73..65860a550 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,56 +1,52 @@ -WholeStageCodegen (4) +WholeStageCodegen (2) Project [amc,pmc] BroadcastNestedLoopJoin - HashAggregate [count] [count(1),amc,count] + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] - CometProject [ws_sold_time_sk,ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] - CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - CometBroadcastExchange [wp_web_page_sk] #4 - CometProject [wp_web_page_sk] - CometFilter [wp_web_page_sk,wp_char_count] - CometScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] + CometHashAggregate [amc,count,count(1)] + CometColumnarExchange #1 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometProject [ws_web_page_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] + CometProject [ws_sold_time_sk,ws_web_page_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] + CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] + CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] + CometBroadcastExchange [hd_demo_sk] #2 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] + CometBroadcastExchange [t_time_sk] #3 + CometProject [t_time_sk] + CometFilter [t_time_sk,t_hour] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] + CometBroadcastExchange [wp_web_page_sk] #4 + CometProject [wp_web_page_sk] + CometFilter [wp_web_page_sk,wp_char_count] + CometScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - HashAggregate [count] [count(1),pmc,count] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange #6 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] - CometProject [ws_sold_time_sk,ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] - CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #7 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - ReusedExchange [wp_web_page_sk] #4 + CometHashAggregate [pmc,count,count(1)] + CometColumnarExchange #6 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometProject [ws_web_page_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] + CometProject [ws_sold_time_sk,ws_web_page_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] + CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] + CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange [t_time_sk] #7 + CometProject [t_time_sk] + CometFilter [t_time_sk,t_hour] + 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 4c0fcd87a..7866655a3 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,45 +1,49 @@ == Physical Plan == -* 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) +* ColumnarToRow (45) ++- CometSort (44) + +- CometColumnarExchange (43) + +- RowToColumnar (42) + +- * HashAggregate (41) + +- * ColumnarToRow (40) + +- CometColumnarExchange (39) + +- RowToColumnar (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 @@ -220,54 +224,66 @@ Functions [1]: [partial_sum(UnscaledValue(cr_net_loss#7))] 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] -(38) Exchange +(38) RowToColumnar 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] -(39) HashAggregate [codegen id : 2] +(39) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(40) ColumnarToRow [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] + +(41) 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))#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] -(40) Exchange +(42) RowToColumnar +Input [4]: [Call_Center#27, Call_Center_Name#28, Manager#29, Returns_Loss#30] + +(43) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] + +(44) CometSort 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] +Arguments: [Call_Center#27, Call_Center_Name#28, Manager#29, Returns_Loss#30], [Returns_Loss#30 DESC NULLS LAST] -(41) Sort [codegen id : 3] +(45) ColumnarToRow [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 = 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) +BroadcastExchange (50) ++- * ColumnarToRow (49) + +- CometProject (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) -(42) Scan parquet spark_catalog.default.date_dim +(46) 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 -(43) CometFilter +(47) 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)) -(44) CometProject +(48) CometProject Input [3]: [d_date_sk#10, d_year#11, d_moy#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(45) ColumnarToRow [codegen id : 1] +(49) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(46) BroadcastExchange +(50) BroadcastExchange Input [1]: [d_date_sk#10] 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 84f1190d7..c55d2962c 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,55 +1,59 @@ WholeStageCodegen (3) - Sort [Returns_Loss] + ColumnarToRow InputAdapter - Exchange [Returns_Loss] #1 - 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 (1) - HashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,cr_net_loss] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status,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 [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status] - CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk] - CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,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 [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss] - CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,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,cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] - CometFilter [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometBroadcastExchange [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] #3 - CometFilter [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] #6 - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk] #7 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #8 - 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 [hd_demo_sk] #9 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] + CometSort [Call_Center,Call_Center_Name,Manager,Returns_Loss] + CometColumnarExchange [Returns_Loss] #1 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,cr_net_loss] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status,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 [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status] + CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,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 [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,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,cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] + CometFilter [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] + CometBroadcastExchange [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] #3 + CometFilter [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_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_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] #6 + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + CometBroadcastExchange [ca_address_sk] #7 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #8 + 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 [hd_demo_sk] #9 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] + 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 e8d482e9b..2825d25af 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,35 +1,39 @@ == Physical Plan == -* 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) +* HashAggregate (35) ++- * ColumnarToRow (34) + +- CometColumnarExchange (33) + +- RowToColumnar (32) + +- * HashAggregate (31) + +- * Project (30) + +- * BroadcastHashJoin Inner BuildRight (29) + :- * Project (27) + : +- * BroadcastHashJoin Inner BuildRight (26) + : :- * 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 (25) + : +- * Filter (24) + : +- * HashAggregate (23) + : +- * ColumnarToRow (22) + : +- CometColumnarExchange (21) + : +- RowToColumnar (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 (28) (1) Scan parquet spark_catalog.default.web_sales @@ -125,60 +129,72 @@ Functions [1]: [partial_avg(UnscaledValue(ws_ext_discount_amt#8))] Aggregate Attributes [2]: [sum#13, count#14] Results [3]: [ws_item_sk#7, sum#15, count#16] -(20) Exchange +(20) RowToColumnar Input [3]: [ws_item_sk#7, sum#15, count#16] -Arguments: hashpartitioning(ws_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(21) HashAggregate [codegen id : 2] +(21) CometColumnarExchange +Input [3]: [ws_item_sk#7, sum#15, count#16] +Arguments: hashpartitioning(ws_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(22) ColumnarToRow [codegen id : 2] +Input [3]: [ws_item_sk#7, sum#15, count#16] + +(23) 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))#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] -(22) Filter [codegen id : 2] +(24) 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) -(23) BroadcastExchange +(25) 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] -(24) BroadcastHashJoin [codegen id : 4] +(26) 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))#18) -(25) Project [codegen id : 4] +(27) 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))#18, ws_item_sk#7] -(26) ReusedExchange [Reuses operator id: 36] +(28) ReusedExchange [Reuses operator id: 40] Output [1]: [d_date_sk#19] -(27) BroadcastHashJoin [codegen id : 4] +(29) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#3] Right keys [1]: [d_date_sk#19] Join type: Inner Join condition: None -(28) Project [codegen id : 4] +(30) 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#19] -(29) HashAggregate [codegen id : 4] +(31) 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#20] Results [1]: [sum#21] -(30) Exchange +(32) RowToColumnar +Input [1]: [sum#21] + +(33) CometColumnarExchange +Input [1]: [sum#21] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(34) ColumnarToRow [codegen id : 5] Input [1]: [sum#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] -(31) HashAggregate [codegen id : 5] +(35) HashAggregate [codegen id : 5] Input [1]: [sum#21] Keys: [] Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] @@ -188,32 +204,32 @@ Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#2))#22,17,2) AS ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (36) -+- * ColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan parquet spark_catalog.default.date_dim (32) +BroadcastExchange (40) ++- * ColumnarToRow (39) + +- CometProject (38) + +- CometFilter (37) + +- CometScan parquet spark_catalog.default.date_dim (36) -(32) Scan parquet spark_catalog.default.date_dim +(36) 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 -(33) CometFilter +(37) 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)) -(34) CometProject +(38) CometProject Input [2]: [d_date_sk#19, d_date#24] Arguments: [d_date_sk#19], [d_date_sk#19] -(35) ColumnarToRow [codegen id : 1] +(39) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] -(36) BroadcastExchange +(40) BroadcastExchange Input [1]: [d_date_sk#19] 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/q92/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/simplified.txt index aa24a049b..df7b1ca66 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,50 +1,54 @@ WholeStageCodegen (5) HashAggregate [sum] [sum(UnscaledValue(ws_ext_discount_amt)),Excess Discount Amount ,sum] - InputAdapter - Exchange #1 - 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))] - ColumnarToRow - InputAdapter - CometProject [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] - CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] - CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk] #3 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + ColumnarToRow + InputAdapter + CometColumnarExchange #1 + RowToColumnar + 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))] + ColumnarToRow + InputAdapter + CometProject [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] + CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk] #3 + CometProject [i_item_sk] + CometFilter [i_item_sk,i_manufact_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + InputAdapter + BroadcastExchange #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] + ColumnarToRow + InputAdapter + CometColumnarExchange [ws_item_sk] #5 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [ws_item_sk,ws_ext_discount_amt] [sum,count,sum,count] + ColumnarToRow + InputAdapter + CometProject [ws_item_sk,ws_ext_discount_amt] + CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #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 (1) - HashAggregate [ws_item_sk,ws_ext_discount_amt] [sum,count,sum,count] - ColumnarToRow - InputAdapter - CometProject [ws_item_sk,ws_ext_discount_amt] - CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/explain.txt index 00ed822f2..4634a45a1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/explain.txt @@ -1,28 +1,28 @@ == Physical Plan == TakeOrderedAndProject (24) +- * HashAggregate (23) - +- Exchange (22) - +- * HashAggregate (21) - +- * Project (20) - +- * BroadcastHashJoin Inner BuildRight (19) - :- * Project (13) - : +- * SortMergeJoin Inner (12) - : :- * Sort (5) - : : +- Exchange (4) - : : +- * ColumnarToRow (3) - : : +- CometProject (2) - : : +- CometScan parquet spark_catalog.default.store_sales (1) - : +- * Sort (11) - : +- Exchange (10) - : +- * ColumnarToRow (9) - : +- CometProject (8) - : +- CometFilter (7) - : +- CometScan parquet spark_catalog.default.store_returns (6) - +- BroadcastExchange (18) - +- * ColumnarToRow (17) - +- CometProject (16) - +- CometFilter (15) - +- CometScan parquet spark_catalog.default.reason (14) + +- * ColumnarToRow (22) + +- CometColumnarExchange (21) + +- RowToColumnar (20) + +- * HashAggregate (19) + +- * ColumnarToRow (18) + +- CometProject (17) + +- CometBroadcastHashJoin (16) + :- CometProject (11) + : +- CometSortMergeJoin (10) + : :- CometSort (4) + : : +- CometColumnarExchange (3) + : : +- CometProject (2) + : : +- CometScan parquet spark_catalog.default.store_sales (1) + : +- CometSort (9) + : +- CometColumnarExchange (8) + : +- CometProject (7) + : +- CometFilter (6) + : +- CometScan parquet spark_catalog.default.store_returns (5) + +- CometBroadcastExchange (15) + +- CometProject (14) + +- CometFilter (13) + +- CometScan parquet spark_catalog.default.reason (12) (1) Scan parquet spark_catalog.default.store_sales @@ -35,97 +35,95 @@ ReadSchema: struct -(7) CometFilter +(6) CometFilter Input [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] Condition : ((isnotnull(sr_item_sk#7) AND isnotnull(sr_ticket_number#9)) AND isnotnull(sr_reason_sk#8)) -(8) CometProject +(7) CometProject Input [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] Arguments: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10], [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -(9) ColumnarToRow [codegen id : 3] -Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] - -(10) Exchange +(8) CometColumnarExchange Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: hashpartitioning(sr_item_sk#7, sr_ticket_number#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(sr_item_sk#7, sr_ticket_number#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(11) Sort [codegen id : 4] +(9) CometSort Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [sr_item_sk#7 ASC NULLS FIRST, sr_ticket_number#9 ASC NULLS FIRST], false, 0 +Arguments: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10], [sr_item_sk#7 ASC NULLS FIRST, sr_ticket_number#9 ASC NULLS FIRST] -(12) SortMergeJoin [codegen id : 6] -Left keys [2]: [ss_item_sk#1, ss_ticket_number#3] -Right keys [2]: [sr_item_sk#7, sr_ticket_number#9] -Join type: Inner -Join condition: None +(10) CometSortMergeJoin +Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] +Right output [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] +Arguments: [ss_item_sk#1, ss_ticket_number#3], [sr_item_sk#7, sr_ticket_number#9], Inner -(13) Project [codegen id : 6] -Output [5]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10] +(11) CometProject Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] +Arguments: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10], [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10] -(14) Scan parquet spark_catalog.default.reason +(12) Scan parquet spark_catalog.default.reason Output [2]: [r_reason_sk#12, r_reason_desc#13] Batched: true Location [not included in comparison]/{warehouse_dir}/reason] PushedFilters: [IsNotNull(r_reason_desc), EqualTo(r_reason_desc,reason 28 ), IsNotNull(r_reason_sk)] ReadSchema: struct -(15) CometFilter +(13) CometFilter Input [2]: [r_reason_sk#12, r_reason_desc#13] Condition : ((isnotnull(r_reason_desc#13) AND (r_reason_desc#13 = reason 28 )) AND isnotnull(r_reason_sk#12)) -(16) CometProject +(14) CometProject Input [2]: [r_reason_sk#12, r_reason_desc#13] Arguments: [r_reason_sk#12], [r_reason_sk#12] -(17) ColumnarToRow [codegen id : 5] -Input [1]: [r_reason_sk#12] - -(18) BroadcastExchange +(15) CometBroadcastExchange Input [1]: [r_reason_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: [r_reason_sk#12] -(19) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_reason_sk#8] -Right keys [1]: [r_reason_sk#12] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [5]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10] +Right output [1]: [r_reason_sk#12] +Arguments: [sr_reason_sk#8], [r_reason_sk#12], Inner, BuildRight -(20) Project [codegen id : 6] -Output [2]: [ss_customer_sk#2, CASE WHEN isnotnull(sr_return_quantity#10) THEN (cast((ss_quantity#4 - sr_return_quantity#10) as decimal(10,0)) * ss_sales_price#5) ELSE (cast(ss_quantity#4 as decimal(10,0)) * ss_sales_price#5) END AS act_sales#14] +(17) CometProject Input [6]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10, r_reason_sk#12] +Arguments: [ss_customer_sk#2, act_sales#14], [ss_customer_sk#2, CASE WHEN isnotnull(sr_return_quantity#10) THEN (cast((ss_quantity#4 - sr_return_quantity#10) as decimal(10,0)) * ss_sales_price#5) ELSE (cast(ss_quantity#4 as decimal(10,0)) * ss_sales_price#5) END AS act_sales#14] -(21) HashAggregate [codegen id : 6] +(18) ColumnarToRow [codegen id : 1] +Input [2]: [ss_customer_sk#2, act_sales#14] + +(19) HashAggregate [codegen id : 1] Input [2]: [ss_customer_sk#2, act_sales#14] Keys [1]: [ss_customer_sk#2] Functions [1]: [partial_sum(act_sales#14)] Aggregate Attributes [2]: [sum#15, isEmpty#16] Results [3]: [ss_customer_sk#2, sum#17, isEmpty#18] -(22) Exchange +(20) RowToColumnar +Input [3]: [ss_customer_sk#2, sum#17, isEmpty#18] + +(21) CometColumnarExchange +Input [3]: [ss_customer_sk#2, sum#17, isEmpty#18] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(22) ColumnarToRow [codegen id : 2] Input [3]: [ss_customer_sk#2, sum#17, isEmpty#18] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(23) HashAggregate [codegen id : 7] +(23) HashAggregate [codegen id : 2] Input [3]: [ss_customer_sk#2, sum#17, isEmpty#18] Keys [1]: [ss_customer_sk#2] Functions [1]: [sum(act_sales#14)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/simplified.txt index a8eb231b7..1adc16371 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/simplified.txt @@ -1,40 +1,28 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] - WholeStageCodegen (7) + WholeStageCodegen (2) HashAggregate [ss_customer_sk,sum,isEmpty] [sum(act_sales),sumsales,sum,isEmpty] - InputAdapter - Exchange [ss_customer_sk] #1 - WholeStageCodegen (6) - HashAggregate [ss_customer_sk,act_sales] [sum,isEmpty,sum,isEmpty] - Project [ss_customer_sk,sr_return_quantity,ss_quantity,ss_sales_price] - BroadcastHashJoin [sr_reason_sk,r_reason_sk] - Project [ss_customer_sk,ss_quantity,ss_sales_price,sr_reason_sk,sr_return_quantity] - SortMergeJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - WholeStageCodegen (2) - Sort [ss_item_sk,ss_ticket_number] - InputAdapter - Exchange [ss_item_sk,ss_ticket_number] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] - InputAdapter - WholeStageCodegen (4) - Sort [sr_item_sk,sr_ticket_number] - InputAdapter - Exchange [sr_item_sk,sr_ticket_number] #3 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometFilter [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometColumnarExchange [ss_customer_sk] #1 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [ss_customer_sk,act_sales] [sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometProject [sr_return_quantity,ss_quantity,ss_sales_price] [ss_customer_sk,act_sales] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,sr_reason_sk,sr_return_quantity,r_reason_sk] + CometProject [ss_customer_sk,ss_quantity,ss_sales_price,sr_reason_sk,sr_return_quantity] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] + CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] + CometColumnarExchange [ss_item_sk,ss_ticket_number] #2 + CometProject [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometSort [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] + CometColumnarExchange [sr_item_sk,sr_ticket_number] #3 + CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] + CometFilter [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometBroadcastExchange [r_reason_sk] #4 CometProject [r_reason_sk] CometFilter [r_reason_sk,r_reason_desc] 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/q94/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/explain.txt index d71f96e15..ff1dd24d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/explain.txt @@ -1,49 +1,51 @@ == Physical Plan == -* HashAggregate (45) -+- Exchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * SortMergeJoin LeftAnti (19) - : : : :- * Project (13) - : : : : +- * SortMergeJoin LeftSemi (12) - : : : : :- * Sort (6) - : : : : : +- Exchange (5) - : : : : : +- * ColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : : +- * Sort (11) - : : : : +- Exchange (10) - : : : : +- * ColumnarToRow (9) - : : : : +- CometProject (8) - : : : : +- CometScan parquet spark_catalog.default.web_sales (7) - : : : +- * Sort (18) - : : : +- Exchange (17) - : : : +- * ColumnarToRow (16) - : : : +- CometProject (15) - : : : +- CometScan parquet spark_catalog.default.web_returns (14) - : : +- BroadcastExchange (24) - : : +- * ColumnarToRow (23) - : : +- CometProject (22) - : : +- CometFilter (21) - : : +- CometScan parquet spark_catalog.default.date_dim (20) - : +- 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.web_site (34) +* HashAggregate (47) ++- * ColumnarToRow (46) + +- CometColumnarExchange (45) + +- RowToColumnar (44) + +- * HashAggregate (43) + +- * HashAggregate (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * SortMergeJoin LeftAnti (19) + : : : :- * Project (13) + : : : : +- * SortMergeJoin LeftSemi (12) + : : : : :- * ColumnarToRow (6) + : : : : : +- CometSort (5) + : : : : : +- CometColumnarExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : : +- * ColumnarToRow (11) + : : : : +- CometSort (10) + : : : : +- CometColumnarExchange (9) + : : : : +- CometProject (8) + : : : : +- CometScan parquet spark_catalog.default.web_sales (7) + : : : +- * ColumnarToRow (18) + : : : +- CometSort (17) + : : : +- CometColumnarExchange (16) + : : : +- CometProject (15) + : : : +- CometScan parquet spark_catalog.default.web_returns (14) + : : +- BroadcastExchange (24) + : : +- * ColumnarToRow (23) + : : +- CometProject (22) + : : +- CometFilter (21) + : : +- CometScan parquet spark_catalog.default.date_dim (20) + : +- 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.web_site (34) (1) Scan parquet spark_catalog.default.web_sales @@ -61,16 +63,16 @@ Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -(4) ColumnarToRow [codegen id : 1] +(4) CometColumnarExchange Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] +Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(5) Exchange +(5) CometSort Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_order_number#5 ASC NULLS FIRST] -(6) Sort [codegen id : 2] +(6) ColumnarToRow [codegen id : 1] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: [ws_order_number#5 ASC NULLS FIRST], false, 0 (7) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] @@ -82,24 +84,24 @@ ReadSchema: struct Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] Arguments: [ws_warehouse_sk#9, ws_order_number#10], [ws_warehouse_sk#9, ws_order_number#10] -(9) ColumnarToRow [codegen id : 3] +(9) CometColumnarExchange Input [2]: [ws_warehouse_sk#9, ws_order_number#10] +Arguments: hashpartitioning(ws_order_number#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(10) Exchange +(10) CometSort Input [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: hashpartitioning(ws_order_number#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: [ws_warehouse_sk#9, ws_order_number#10], [ws_order_number#10 ASC NULLS FIRST] -(11) Sort [codegen id : 4] +(11) ColumnarToRow [codegen id : 2] Input [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: [ws_order_number#10 ASC NULLS FIRST], false, 0 -(12) SortMergeJoin [codegen id : 5] +(12) SortMergeJoin [codegen id : 3] Left keys [1]: [ws_order_number#5] Right keys [1]: [ws_order_number#10] Join type: LeftSemi Join condition: NOT (ws_warehouse_sk#4 = ws_warehouse_sk#9) -(13) Project [codegen id : 5] +(13) Project [codegen id : 3] Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] @@ -113,18 +115,18 @@ ReadSchema: struct Input [2]: [wr_order_number#12, wr_returned_date_sk#13] Arguments: [wr_order_number#12], [wr_order_number#12] -(16) ColumnarToRow [codegen id : 6] +(16) CometColumnarExchange Input [1]: [wr_order_number#12] +Arguments: hashpartitioning(wr_order_number#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(17) Exchange +(17) CometSort Input [1]: [wr_order_number#12] -Arguments: hashpartitioning(wr_order_number#12, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: [wr_order_number#12], [wr_order_number#12 ASC NULLS FIRST] -(18) Sort [codegen id : 7] +(18) ColumnarToRow [codegen id : 4] Input [1]: [wr_order_number#12] -Arguments: [wr_order_number#12 ASC NULLS FIRST], false, 0 -(19) SortMergeJoin [codegen id : 11] +(19) SortMergeJoin [codegen id : 8] Left keys [1]: [ws_order_number#5] Right keys [1]: [wr_order_number#12] Join type: LeftAnti @@ -145,20 +147,20 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-01)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(23) ColumnarToRow [codegen id : 8] +(23) ColumnarToRow [codegen id : 5] Input [1]: [d_date_sk#14] (24) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(25) BroadcastHashJoin [codegen id : 11] +(25) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ws_ship_date_sk#1] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(26) Project [codegen id : 11] +(26) Project [codegen id : 8] Output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#14] @@ -177,20 +179,20 @@ Condition : ((isnotnull(ca_state#17) AND (ca_state#17 = IL)) AND isnotnull(ca_ad Input [2]: [ca_address_sk#16, ca_state#17] Arguments: [ca_address_sk#16], [ca_address_sk#16] -(30) ColumnarToRow [codegen id : 9] +(30) ColumnarToRow [codegen id : 6] Input [1]: [ca_address_sk#16] (31) BroadcastExchange Input [1]: [ca_address_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(32) BroadcastHashJoin [codegen id : 11] +(32) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ws_ship_addr_sk#2] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(33) Project [codegen id : 11] +(33) Project [codegen id : 8] Output [4]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#16] @@ -209,49 +211,55 @@ Condition : ((isnotnull(web_company_name#19) AND (web_company_name#19 = pri Input [2]: [web_site_sk#18, web_company_name#19] Arguments: [web_site_sk#18], [web_site_sk#18] -(37) ColumnarToRow [codegen id : 10] +(37) ColumnarToRow [codegen id : 7] Input [1]: [web_site_sk#18] (38) BroadcastExchange Input [1]: [web_site_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(39) BroadcastHashJoin [codegen id : 11] +(39) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ws_web_site_sk#3] Right keys [1]: [web_site_sk#18] Join type: Inner Join condition: None -(40) Project [codegen id : 11] +(40) Project [codegen id : 8] Output [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [5]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#18] -(41) HashAggregate [codegen id : 11] +(41) HashAggregate [codegen id : 8] Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Keys [1]: [ws_order_number#5] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#20, sum(UnscaledValue(ws_net_profit#7))#21] Results [3]: [ws_order_number#5, sum#22, sum#23] -(42) HashAggregate [codegen id : 11] +(42) HashAggregate [codegen id : 8] Input [3]: [ws_order_number#5, sum#22, sum#23] Keys [1]: [ws_order_number#5] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#20, sum(UnscaledValue(ws_net_profit#7))#21] Results [3]: [ws_order_number#5, sum#22, sum#23] -(43) HashAggregate [codegen id : 11] +(43) HashAggregate [codegen id : 8] Input [3]: [ws_order_number#5, sum#22, sum#23] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#20, sum(UnscaledValue(ws_net_profit#7))#21, count(ws_order_number#5)#24] Results [3]: [sum#22, sum#23, count#25] -(44) Exchange +(44) RowToColumnar Input [3]: [sum#22, sum#23, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(45) HashAggregate [codegen id : 12] +(45) CometColumnarExchange +Input [3]: [sum#22, sum#23, count#25] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(46) ColumnarToRow [codegen id : 9] +Input [3]: [sum#22, sum#23, count#25] + +(47) HashAggregate [codegen id : 9] Input [3]: [sum#22, sum#23, count#25] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/simplified.txt index d54c9e0c9..3f324f9f2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/simplified.txt @@ -1,74 +1,70 @@ -WholeStageCodegen (12) +WholeStageCodegen (9) HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] - InputAdapter - Exchange #1 - WholeStageCodegen (11) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] - Project [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_date_sk,d_date_sk] - SortMergeJoin [ws_order_number,wr_order_number] - InputAdapter - WholeStageCodegen (5) - Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] - InputAdapter - WholeStageCodegen (2) - Sort [ws_order_number] - InputAdapter - Exchange [ws_order_number] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometColumnarExchange #1 + RowToColumnar + WholeStageCodegen (8) + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] + Project [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_date_sk,d_date_sk] + SortMergeJoin [ws_order_number,wr_order_number] + InputAdapter + WholeStageCodegen (3) + Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometSort [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometColumnarExchange [ws_order_number] #2 CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (4) - Sort [ws_order_number] - InputAdapter - Exchange [ws_order_number] #3 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter + InputAdapter + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometSort [ws_warehouse_sk,ws_order_number] + CometColumnarExchange [ws_order_number] #3 CometProject [ws_warehouse_sk,ws_order_number] CometScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (7) - Sort [wr_order_number] - InputAdapter - Exchange [wr_order_number] #4 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter + InputAdapter + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometSort [wr_order_number] + CometColumnarExchange [wr_order_number] #4 CometProject [wr_order_number] CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (8) + BroadcastExchange #6 + WholeStageCodegen (6) ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) + BroadcastExchange #7 + WholeStageCodegen (7) ColumnarToRow InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometProject [web_site_sk] - CometFilter [web_site_sk,web_company_name] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] + CometProject [web_site_sk] + CometFilter [web_site_sk,web_company_name] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/explain.txt index c8cdce055..f58d42056 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/explain.txt @@ -1,62 +1,67 @@ == Physical Plan == -* HashAggregate (58) -+- Exchange (57) - +- * HashAggregate (56) - +- * HashAggregate (55) - +- * HashAggregate (54) - +- * Project (53) - +- * BroadcastHashJoin Inner BuildRight (52) - :- * Project (46) - : +- * BroadcastHashJoin Inner BuildRight (45) - : :- * Project (39) - : : +- * BroadcastHashJoin Inner BuildRight (38) - : : :- * SortMergeJoin LeftSemi (32) - : : : :- * SortMergeJoin LeftSemi (17) - : : : : :- * Sort (6) - : : : : : +- Exchange (5) - : : : : : +- * ColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : : +- * Project (16) - : : : : +- * SortMergeJoin Inner (15) - : : : : :- * Sort (12) - : : : : : +- Exchange (11) - : : : : : +- * ColumnarToRow (10) - : : : : : +- CometProject (9) - : : : : : +- CometFilter (8) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (7) - : : : : +- * Sort (14) - : : : : +- ReusedExchange (13) - : : : +- * Project (31) - : : : +- * SortMergeJoin Inner (30) - : : : :- * Sort (23) - : : : : +- Exchange (22) - : : : : +- * ColumnarToRow (21) - : : : : +- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan parquet spark_catalog.default.web_returns (18) - : : : +- * Project (29) - : : : +- * SortMergeJoin Inner (28) - : : : :- * Sort (25) - : : : : +- ReusedExchange (24) - : : : +- * Sort (27) - : : : +- ReusedExchange (26) - : : +- BroadcastExchange (37) - : : +- * ColumnarToRow (36) - : : +- CometProject (35) - : : +- CometFilter (34) - : : +- CometScan parquet spark_catalog.default.date_dim (33) - : +- BroadcastExchange (44) - : +- * ColumnarToRow (43) - : +- CometProject (42) - : +- CometFilter (41) - : +- CometScan parquet spark_catalog.default.customer_address (40) - +- BroadcastExchange (51) - +- * ColumnarToRow (50) - +- CometProject (49) - +- CometFilter (48) - +- CometScan parquet spark_catalog.default.web_site (47) +* HashAggregate (63) ++- * ColumnarToRow (62) + +- CometColumnarExchange (61) + +- RowToColumnar (60) + +- * HashAggregate (59) + +- * HashAggregate (58) + +- * HashAggregate (57) + +- * Project (56) + +- * BroadcastHashJoin Inner BuildRight (55) + :- * Project (49) + : +- * BroadcastHashJoin Inner BuildRight (48) + : :- * Project (42) + : : +- * BroadcastHashJoin Inner BuildRight (41) + : : :- * SortMergeJoin LeftSemi (35) + : : : :- * SortMergeJoin LeftSemi (18) + : : : : :- * ColumnarToRow (6) + : : : : : +- CometSort (5) + : : : : : +- CometColumnarExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : : +- * Project (17) + : : : : +- * SortMergeJoin Inner (16) + : : : : :- * ColumnarToRow (12) + : : : : : +- CometSort (11) + : : : : : +- CometColumnarExchange (10) + : : : : : +- CometProject (9) + : : : : : +- CometFilter (8) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (7) + : : : : +- * ColumnarToRow (15) + : : : : +- CometSort (14) + : : : : +- ReusedExchange (13) + : : : +- * Project (34) + : : : +- * SortMergeJoin Inner (33) + : : : :- * ColumnarToRow (24) + : : : : +- CometSort (23) + : : : : +- CometColumnarExchange (22) + : : : : +- CometProject (21) + : : : : +- CometFilter (20) + : : : : +- CometScan parquet spark_catalog.default.web_returns (19) + : : : +- * Project (32) + : : : +- * SortMergeJoin Inner (31) + : : : :- * ColumnarToRow (27) + : : : : +- CometSort (26) + : : : : +- ReusedExchange (25) + : : : +- * ColumnarToRow (30) + : : : +- CometSort (29) + : : : +- ReusedExchange (28) + : : +- BroadcastExchange (40) + : : +- * ColumnarToRow (39) + : : +- CometProject (38) + : : +- CometFilter (37) + : : +- CometScan parquet spark_catalog.default.date_dim (36) + : +- BroadcastExchange (47) + : +- * ColumnarToRow (46) + : +- CometProject (45) + : +- CometFilter (44) + : +- CometScan parquet spark_catalog.default.customer_address (43) + +- BroadcastExchange (54) + +- * ColumnarToRow (53) + +- CometProject (52) + +- CometFilter (51) + +- CometScan parquet spark_catalog.default.web_site (50) (1) Scan parquet spark_catalog.default.web_sales @@ -74,16 +79,16 @@ Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -(4) ColumnarToRow [codegen id : 1] +(4) CometColumnarExchange Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] +Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(5) Exchange +(5) CometSort Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_order_number#4 ASC NULLS FIRST] -(6) Sort [codegen id : 2] +(6) ColumnarToRow [codegen id : 1] Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Arguments: [ws_order_number#4 ASC NULLS FIRST], false, 0 (7) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] @@ -100,228 +105,243 @@ Condition : (isnotnull(ws_order_number#9) AND isnotnull(ws_warehouse_sk#8)) Input [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_warehouse_sk#8, ws_order_number#9] -(10) ColumnarToRow [codegen id : 3] +(10) CometColumnarExchange Input [2]: [ws_warehouse_sk#8, ws_order_number#9] +Arguments: hashpartitioning(ws_order_number#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(11) Exchange +(11) CometSort Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: hashpartitioning(ws_order_number#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_order_number#9 ASC NULLS FIRST] -(12) Sort [codegen id : 4] +(12) ColumnarToRow [codegen id : 2] Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: [ws_order_number#9 ASC NULLS FIRST], false, 0 -(13) ReusedExchange [Reuses operator id: 11] +(13) ReusedExchange [Reuses operator id: 10] Output [2]: [ws_warehouse_sk#11, ws_order_number#12] -(14) Sort [codegen id : 6] +(14) CometSort Input [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_order_number#12 ASC NULLS FIRST], false, 0 +Arguments: [ws_warehouse_sk#11, ws_order_number#12], [ws_order_number#12 ASC NULLS FIRST] -(15) SortMergeJoin [codegen id : 7] +(15) ColumnarToRow [codegen id : 3] +Input [2]: [ws_warehouse_sk#11, ws_order_number#12] + +(16) SortMergeJoin [codegen id : 4] Left keys [1]: [ws_order_number#9] Right keys [1]: [ws_order_number#12] Join type: Inner Join condition: NOT (ws_warehouse_sk#8 = ws_warehouse_sk#11) -(16) Project [codegen id : 7] +(17) Project [codegen id : 4] Output [1]: [ws_order_number#9] Input [4]: [ws_warehouse_sk#8, ws_order_number#9, ws_warehouse_sk#11, ws_order_number#12] -(17) SortMergeJoin [codegen id : 8] +(18) SortMergeJoin [codegen id : 5] Left keys [1]: [ws_order_number#4] Right keys [1]: [ws_order_number#9] Join type: LeftSemi Join condition: None -(18) Scan parquet spark_catalog.default.web_returns +(19) Scan parquet spark_catalog.default.web_returns Output [2]: [wr_order_number#13, wr_returned_date_sk#14] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number)] ReadSchema: struct -(19) CometFilter +(20) CometFilter Input [2]: [wr_order_number#13, wr_returned_date_sk#14] Condition : isnotnull(wr_order_number#13) -(20) CometProject +(21) CometProject Input [2]: [wr_order_number#13, wr_returned_date_sk#14] Arguments: [wr_order_number#13], [wr_order_number#13] -(21) ColumnarToRow [codegen id : 9] +(22) CometColumnarExchange Input [1]: [wr_order_number#13] +Arguments: hashpartitioning(wr_order_number#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) Exchange +(23) CometSort Input [1]: [wr_order_number#13] -Arguments: hashpartitioning(wr_order_number#13, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: [wr_order_number#13], [wr_order_number#13 ASC NULLS FIRST] -(23) Sort [codegen id : 10] +(24) ColumnarToRow [codegen id : 6] Input [1]: [wr_order_number#13] -Arguments: [wr_order_number#13 ASC NULLS FIRST], false, 0 -(24) ReusedExchange [Reuses operator id: 11] +(25) ReusedExchange [Reuses operator id: 10] Output [2]: [ws_warehouse_sk#15, ws_order_number#16] -(25) Sort [codegen id : 12] +(26) CometSort Input [2]: [ws_warehouse_sk#15, ws_order_number#16] -Arguments: [ws_order_number#16 ASC NULLS FIRST], false, 0 +Arguments: [ws_warehouse_sk#15, ws_order_number#16], [ws_order_number#16 ASC NULLS FIRST] -(26) ReusedExchange [Reuses operator id: 11] +(27) ColumnarToRow [codegen id : 7] +Input [2]: [ws_warehouse_sk#15, ws_order_number#16] + +(28) ReusedExchange [Reuses operator id: 10] Output [2]: [ws_warehouse_sk#17, ws_order_number#18] -(27) Sort [codegen id : 14] +(29) CometSort +Input [2]: [ws_warehouse_sk#17, ws_order_number#18] +Arguments: [ws_warehouse_sk#17, ws_order_number#18], [ws_order_number#18 ASC NULLS FIRST] + +(30) ColumnarToRow [codegen id : 8] Input [2]: [ws_warehouse_sk#17, ws_order_number#18] -Arguments: [ws_order_number#18 ASC NULLS FIRST], false, 0 -(28) SortMergeJoin [codegen id : 15] +(31) SortMergeJoin [codegen id : 9] Left keys [1]: [ws_order_number#16] Right keys [1]: [ws_order_number#18] Join type: Inner Join condition: NOT (ws_warehouse_sk#15 = ws_warehouse_sk#17) -(29) Project [codegen id : 15] +(32) Project [codegen id : 9] Output [1]: [ws_order_number#16] Input [4]: [ws_warehouse_sk#15, ws_order_number#16, ws_warehouse_sk#17, ws_order_number#18] -(30) SortMergeJoin [codegen id : 16] +(33) SortMergeJoin [codegen id : 10] Left keys [1]: [wr_order_number#13] Right keys [1]: [ws_order_number#16] Join type: Inner Join condition: None -(31) Project [codegen id : 16] +(34) Project [codegen id : 10] Output [1]: [wr_order_number#13] Input [2]: [wr_order_number#13, ws_order_number#16] -(32) SortMergeJoin [codegen id : 20] +(35) SortMergeJoin [codegen id : 14] Left keys [1]: [ws_order_number#4] Right keys [1]: [wr_order_number#13] Join type: LeftSemi Join condition: None -(33) Scan parquet spark_catalog.default.date_dim +(36) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#19, d_date#20] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] ReadSchema: struct -(34) CometFilter +(37) CometFilter Input [2]: [d_date_sk#19, d_date#20] Condition : (((isnotnull(d_date#20) AND (d_date#20 >= 1999-02-01)) AND (d_date#20 <= 1999-04-02)) AND isnotnull(d_date_sk#19)) -(35) CometProject +(38) CometProject Input [2]: [d_date_sk#19, d_date#20] Arguments: [d_date_sk#19], [d_date_sk#19] -(36) ColumnarToRow [codegen id : 17] +(39) ColumnarToRow [codegen id : 11] Input [1]: [d_date_sk#19] -(37) BroadcastExchange +(40) BroadcastExchange Input [1]: [d_date_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(38) BroadcastHashJoin [codegen id : 20] +(41) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ws_ship_date_sk#1] Right keys [1]: [d_date_sk#19] Join type: Inner Join condition: None -(39) Project [codegen id : 20] +(42) Project [codegen id : 14] Output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, d_date_sk#19] -(40) Scan parquet spark_catalog.default.customer_address +(43) Scan parquet spark_catalog.default.customer_address Output [2]: [ca_address_sk#21, ca_state#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,IL), IsNotNull(ca_address_sk)] ReadSchema: struct -(41) CometFilter +(44) CometFilter Input [2]: [ca_address_sk#21, ca_state#22] Condition : ((isnotnull(ca_state#22) AND (ca_state#22 = IL)) AND isnotnull(ca_address_sk#21)) -(42) CometProject +(45) CometProject Input [2]: [ca_address_sk#21, ca_state#22] Arguments: [ca_address_sk#21], [ca_address_sk#21] -(43) ColumnarToRow [codegen id : 18] +(46) ColumnarToRow [codegen id : 12] Input [1]: [ca_address_sk#21] -(44) BroadcastExchange +(47) BroadcastExchange Input [1]: [ca_address_sk#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(45) BroadcastHashJoin [codegen id : 20] +(48) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ws_ship_addr_sk#2] Right keys [1]: [ca_address_sk#21] Join type: Inner Join condition: None -(46) Project [codegen id : 20] +(49) Project [codegen id : 14] Output [4]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ca_address_sk#21] -(47) Scan parquet spark_catalog.default.web_site +(50) Scan parquet spark_catalog.default.web_site Output [2]: [web_site_sk#23, web_company_name#24] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] ReadSchema: struct -(48) CometFilter +(51) CometFilter Input [2]: [web_site_sk#23, web_company_name#24] Condition : ((isnotnull(web_company_name#24) AND (web_company_name#24 = pri )) AND isnotnull(web_site_sk#23)) -(49) CometProject +(52) CometProject Input [2]: [web_site_sk#23, web_company_name#24] Arguments: [web_site_sk#23], [web_site_sk#23] -(50) ColumnarToRow [codegen id : 19] +(53) ColumnarToRow [codegen id : 13] Input [1]: [web_site_sk#23] -(51) BroadcastExchange +(54) BroadcastExchange Input [1]: [web_site_sk#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(52) BroadcastHashJoin [codegen id : 20] +(55) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ws_web_site_sk#3] Right keys [1]: [web_site_sk#23] Join type: Inner Join condition: None -(53) Project [codegen id : 20] +(56) Project [codegen id : 14] Output [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Input [5]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, web_site_sk#23] -(54) HashAggregate [codegen id : 20] +(57) HashAggregate [codegen id : 14] Input [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Keys [1]: [ws_order_number#4] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#5)), partial_sum(UnscaledValue(ws_net_profit#6))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#25, sum(UnscaledValue(ws_net_profit#6))#26] Results [3]: [ws_order_number#4, sum#27, sum#28] -(55) HashAggregate [codegen id : 20] +(58) HashAggregate [codegen id : 14] Input [3]: [ws_order_number#4, sum#27, sum#28] Keys [1]: [ws_order_number#4] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#25, sum(UnscaledValue(ws_net_profit#6))#26] Results [3]: [ws_order_number#4, sum#27, sum#28] -(56) HashAggregate [codegen id : 20] +(59) HashAggregate [codegen id : 14] Input [3]: [ws_order_number#4, sum#27, sum#28] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6)), partial_count(distinct ws_order_number#4)] Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#25, sum(UnscaledValue(ws_net_profit#6))#26, count(ws_order_number#4)#29] Results [3]: [sum#27, sum#28, count#30] -(57) Exchange +(60) RowToColumnar +Input [3]: [sum#27, sum#28, count#30] + +(61) CometColumnarExchange +Input [3]: [sum#27, sum#28, count#30] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(62) ColumnarToRow [codegen id : 15] Input [3]: [sum#27, sum#28, count#30] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(58) HashAggregate [codegen id : 21] +(63) HashAggregate [codegen id : 15] Input [3]: [sum#27, sum#28, count#30] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#5)), sum(UnscaledValue(ws_net_profit#6)), count(distinct ws_order_number#4)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/simplified.txt index 178c25f66..2c736e6e5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/simplified.txt @@ -1,102 +1,101 @@ -WholeStageCodegen (21) +WholeStageCodegen (15) HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] - InputAdapter - Exchange #1 - WholeStageCodegen (20) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] - Project [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_date_sk,d_date_sk] - SortMergeJoin [ws_order_number,wr_order_number] - InputAdapter - WholeStageCodegen (8) - SortMergeJoin [ws_order_number,ws_order_number] - InputAdapter - WholeStageCodegen (2) - Sort [ws_order_number] - InputAdapter - Exchange [ws_order_number] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometColumnarExchange #1 + RowToColumnar + WholeStageCodegen (14) + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] + Project [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_date_sk,d_date_sk] + SortMergeJoin [ws_order_number,wr_order_number] + InputAdapter + WholeStageCodegen (5) + SortMergeJoin [ws_order_number,ws_order_number] + InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometSort [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometColumnarExchange [ws_order_number] #2 CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (7) - Project [ws_order_number] - SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] - InputAdapter - WholeStageCodegen (4) - Sort [ws_order_number] - InputAdapter - Exchange [ws_order_number] #3 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter + InputAdapter + WholeStageCodegen (4) + Project [ws_order_number] + SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + InputAdapter + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometSort [ws_warehouse_sk,ws_order_number] + CometColumnarExchange [ws_order_number] #3 CometProject [ws_warehouse_sk,ws_order_number] CometFilter [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (6) - Sort [ws_order_number] - InputAdapter - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - InputAdapter - WholeStageCodegen (16) - Project [wr_order_number] - SortMergeJoin [wr_order_number,ws_order_number] - InputAdapter - WholeStageCodegen (10) - Sort [wr_order_number] - InputAdapter - Exchange [wr_order_number] #4 - WholeStageCodegen (9) - ColumnarToRow - InputAdapter + InputAdapter + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometSort [ws_warehouse_sk,ws_order_number] + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + InputAdapter + WholeStageCodegen (10) + Project [wr_order_number] + SortMergeJoin [wr_order_number,ws_order_number] + InputAdapter + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometSort [wr_order_number] + CometColumnarExchange [wr_order_number] #4 CometProject [wr_order_number] CometFilter [wr_order_number,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] - InputAdapter - WholeStageCodegen (15) - Project [ws_order_number] - SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] - InputAdapter - WholeStageCodegen (12) - Sort [ws_order_number] - InputAdapter - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - InputAdapter - WholeStageCodegen (14) - Sort [ws_order_number] - InputAdapter - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + InputAdapter + WholeStageCodegen (9) + Project [ws_order_number] + SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + InputAdapter + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometSort [ws_warehouse_sk,ws_order_number] + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + InputAdapter + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometSort [ws_warehouse_sk,ws_order_number] + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (17) + BroadcastExchange #6 + WholeStageCodegen (12) ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (18) + BroadcastExchange #7 + WholeStageCodegen (13) ColumnarToRow InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (19) - ColumnarToRow - InputAdapter - CometProject [web_site_sk] - CometFilter [web_site_sk,web_company_name] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] + CometProject [web_site_sk] + CometFilter [web_site_sk,web_company_name] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] 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 4bc24750f..6f9d1a3f2 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,7 +1,7 @@ == Physical Plan == -* HashAggregate (25) -+- Exchange (24) - +- * ColumnarToRow (23) +* ColumnarToRow (25) ++- CometHashAggregate (24) + +- CometColumnarExchange (23) +- CometHashAggregate (22) +- CometProject (21) +- CometBroadcastHashJoin (20) @@ -129,17 +129,15 @@ Input: [] Keys: [] Functions [1]: [partial_count(1)] -(23) ColumnarToRow [codegen id : 1] +(23) CometColumnarExchange Input [1]: [count#12] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(24) Exchange -Input [1]: [count#12] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] - -(25) HashAggregate [codegen id : 2] +(24) CometHashAggregate Input [1]: [count#12] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#13] -Results [1]: [count(1)#13 AS count(1)#14] + +(25) ColumnarToRow [codegen id : 1] +Input [1]: [count(1)#13] 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 e9d33a7f6..b8b3e211d 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,29 +1,27 @@ -WholeStageCodegen (2) - HashAggregate [count] [count(1),count(1),count] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] - CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_store_name] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] + CometHashAggregate [count(1),count,count(1)] + CometColumnarExchange #1 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometBroadcastExchange [hd_demo_sk] #2 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] + CometBroadcastExchange [t_time_sk] #3 + CometProject [t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + CometBroadcastExchange [s_store_sk] #4 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_store_name] + 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 7508405a8..6c90e74cb 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,30 +1,31 @@ == Physical Plan == -* 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) +* HashAggregate (27) ++- * ColumnarToRow (26) + +- CometColumnarExchange (25) + +- RowToColumnar (24) + +- * HashAggregate (23) + +- * ColumnarToRow (22) + +- CometProject (21) + +- CometSortMergeJoin (20) + :- CometSort (11) + : +- CometHashAggregate (10) + : +- CometColumnarExchange (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) + +- CometSort (19) + +- CometHashAggregate (18) + +- CometColumnarExchange (17) + +- CometHashAggregate (16) + +- CometProject (15) + +- CometBroadcastHashJoin (14) + :- CometScan parquet spark_catalog.default.catalog_sales (12) + +- ReusedExchange (13) (1) Scan parquet spark_catalog.default.store_sales @@ -67,88 +68,86 @@ Input [2]: [ss_item_sk#1, ss_customer_sk#2] Keys [2]: [ss_customer_sk#2, ss_item_sk#1] Functions: [] -(9) ColumnarToRow [codegen id : 1] +(9) CometColumnarExchange Input [2]: [ss_customer_sk#2, ss_item_sk#1] +Arguments: hashpartitioning(ss_customer_sk#2, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=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] - -(11) HashAggregate [codegen id : 2] +(10) CometHashAggregate 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#7, ss_item_sk#1 AS item_sk#8] -(12) Sort [codegen id : 2] +(11) CometSort Input [2]: [customer_sk#7, item_sk#8] -Arguments: [customer_sk#7 ASC NULLS FIRST, item_sk#8 ASC NULLS FIRST], false, 0 +Arguments: [customer_sk#7, item_sk#8], [customer_sk#7 ASC NULLS FIRST, item_sk#8 ASC NULLS FIRST] -(13) Scan parquet spark_catalog.default.catalog_sales +(12) 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#11), dynamicpruningexpression(cs_sold_date_sk#11 IN dynamicpruning#12)] ReadSchema: struct -(14) ReusedExchange [Reuses operator id: 5] +(13) ReusedExchange [Reuses operator id: 5] Output [1]: [d_date_sk#13] -(15) CometBroadcastHashJoin +(14) 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 -(16) CometProject +(15) 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] -(17) CometHashAggregate +(16) CometHashAggregate Input [2]: [cs_bill_customer_sk#9, cs_item_sk#10] Keys [2]: [cs_bill_customer_sk#9, cs_item_sk#10] Functions: [] -(18) ColumnarToRow [codegen id : 3] +(17) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=2] -(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] - -(20) HashAggregate [codegen id : 4] +(18) 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#9 AS customer_sk#14, cs_item_sk#10 AS item_sk#15] -(21) Sort [codegen id : 4] +(19) CometSort Input [2]: [customer_sk#14, item_sk#15] -Arguments: [customer_sk#14 ASC NULLS FIRST, item_sk#15 ASC NULLS FIRST], false, 0 +Arguments: [customer_sk#14, item_sk#15], [customer_sk#14 ASC NULLS FIRST, item_sk#15 ASC NULLS FIRST] -(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) CometSortMergeJoin +Left output [2]: [customer_sk#7, item_sk#8] +Right output [2]: [customer_sk#14, item_sk#15] +Arguments: [customer_sk#7, item_sk#8], [customer_sk#14, item_sk#15], FullOuter -(23) Project [codegen id : 5] -Output [2]: [customer_sk#7, customer_sk#14] +(21) CometProject Input [4]: [customer_sk#7, item_sk#8, customer_sk#14, item_sk#15] +Arguments: [customer_sk#7, customer_sk#14], [customer_sk#7, customer_sk#14] + +(22) ColumnarToRow [codegen id : 1] +Input [2]: [customer_sk#7, customer_sk#14] -(24) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 1] Input [2]: [customer_sk#7, customer_sk#14] Keys: [] 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] -(25) Exchange +(24) RowToColumnar +Input [3]: [sum#19, sum#20, sum#21] + +(25) CometColumnarExchange +Input [3]: [sum#19, sum#20, sum#21] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(26) ColumnarToRow [codegen id : 2] Input [3]: [sum#19, sum#20, sum#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] -(26) HashAggregate [codegen id : 6] +(27) HashAggregate [codegen id : 2] Input [3]: [sum#19, sum#20, sum#21] Keys: [] 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)] @@ -158,35 +157,35 @@ Results [3]: [sum(CASE WHEN (isnotnull(customer_sk#7) AND isnull(customer_sk#14) ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (31) -+- * ColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan parquet spark_catalog.default.date_dim (27) +BroadcastExchange (32) ++- * ColumnarToRow (31) + +- CometProject (30) + +- CometFilter (29) + +- CometScan parquet spark_catalog.default.date_dim (28) -(27) Scan parquet spark_catalog.default.date_dim +(28) 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 -(28) CometFilter +(29) 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)) -(29) CometProject +(30) CometProject Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(30) ColumnarToRow [codegen id : 1] +(31) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(31) BroadcastExchange +(32) 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 = 13 Hosting Expression = cs_sold_date_sk#11 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 12 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 41f3e579f..5e09fecd5 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,48 +1,40 @@ -WholeStageCodegen (6) +WholeStageCodegen (2) 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 (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 (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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ss_customer_sk,ss_item_sk] - CometProject [ss_item_sk,ss_customer_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,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 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - 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] #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometHashAggregate [cs_bill_customer_sk,cs_item_sk] - CometProject [cs_bill_customer_sk,cs_item_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,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 - ReusedExchange [d_date_sk] #4 + ColumnarToRow + InputAdapter + CometColumnarExchange #1 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [customer_sk,customer_sk] [sum,sum,sum,sum,sum,sum] + ColumnarToRow + InputAdapter + CometProject [customer_sk,customer_sk] + CometSortMergeJoin [customer_sk,item_sk,customer_sk,item_sk] + CometSort [customer_sk,item_sk] + CometHashAggregate [customer_sk,item_sk,ss_customer_sk,ss_item_sk] + CometColumnarExchange [ss_customer_sk,ss_item_sk] #2 + CometHashAggregate [ss_customer_sk,ss_item_sk] + CometProject [ss_item_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,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 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometSort [customer_sk,item_sk] + CometHashAggregate [customer_sk,item_sk,cs_bill_customer_sk,cs_item_sk] + CometColumnarExchange [cs_bill_customer_sk,cs_item_sk] #5 + CometHashAggregate [cs_bill_customer_sk,cs_item_sk] + CometProject [cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,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 + 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 af9d0026a..cb1e99486 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,28 +1,34 @@ == Physical Plan == -* 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) +* ColumnarToRow (30) ++- CometProject (29) + +- CometSort (28) + +- CometColumnarExchange (27) + +- RowToColumnar (26) + +- * Project (25) + +- Window (24) + +- * ColumnarToRow (23) + +- CometSort (22) + +- CometColumnarExchange (21) + +- RowToColumnar (20) + +- * HashAggregate (19) + +- * ColumnarToRow (18) + +- CometColumnarExchange (17) + +- RowToColumnar (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 @@ -99,74 +105,92 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] 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] -(16) Exchange +(16) RowToColumnar 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] -(17) HashAggregate [codegen id : 2] +(17) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(18) ColumnarToRow [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] + +(19) 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))#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] -(18) Exchange +(20) RowToColumnar +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17, i_item_id#6] + +(21) CometColumnarExchange 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] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(19) Sort [codegen id : 3] +(22) CometSort 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 +Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17, i_item_id#6], [i_class#9 ASC NULLS FIRST] -(20) Window +(23) ColumnarToRow [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] + +(24) 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] -(21) Project [codegen id : 4] +(25) 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] -(22) Exchange +(26) RowToColumnar 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] -(23) Sort [codegen id : 5] +(27) CometColumnarExchange 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 +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, CometColumnarShuffle, [plan_id=3] -(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] +(28) CometSort 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_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, revenueratio#19, i_item_id#6], [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] + +(29) CometProject +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_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, revenueratio#19], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, revenueratio#19] + +(30) ColumnarToRow [codegen id : 5] +Input [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 = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * ColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan parquet spark_catalog.default.date_dim (25) +BroadcastExchange (35) ++- * ColumnarToRow (34) + +- CometProject (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.date_dim (31) -(25) Scan parquet spark_catalog.default.date_dim +(31) 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 -(26) CometFilter +(32) 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)) -(27) CometProject +(33) CometProject Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(28) ColumnarToRow [codegen id : 1] +(34) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(29) BroadcastExchange +(35) BroadcastExchange Input [1]: [d_date_sk#11] 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 a92d99be4..60e9051c7 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,42 +1,48 @@ 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 (4) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (3) - Sort [i_class] - InputAdapter - Exchange [i_class] #2 - 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 (1) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] + CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 + RowToColumnar + 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 (3) + ColumnarToRow + InputAdapter + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometColumnarExchange [i_class] #2 + RowToColumnar + 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] ColumnarToRow InputAdapter - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_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 #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_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 #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + 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 a6282e50c..100603c4e 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,32 +1,34 @@ == Physical Plan == -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) +TakeOrderedAndProject (30) ++- * HashAggregate (29) + +- * ColumnarToRow (28) + +- CometColumnarExchange (27) + +- RowToColumnar (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 @@ -150,18 +152,24 @@ Functions [5]: [partial_sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) < 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] -(26) Exchange +(26) RowToColumnar 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=1] -(27) HashAggregate [codegen id : 2] +(27) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(28) ColumnarToRow [codegen id : 2] +Input [8]: [_groupingexpression#14, sm_type#9, cc_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] + +(29) 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] -(28) TakeOrderedAndProject +(30) 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 25ff3a2fe..26e56c432 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,32 +1,34 @@ 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 (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 (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] - ColumnarToRow - InputAdapter - CometProject [w_warehouse_name] [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,_groupingexpression] - CometBroadcastHashJoin [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name,d_date_sk] - CometProject [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name] - CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_call_center_sk,cc_name] - CometProject [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type] - CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] - CometProject [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name] - CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name] - CometFilter [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_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] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 - CometFilter [sm_ship_mode_sk,sm_type] - CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange [cc_call_center_sk,cc_name] #4 - CometFilter [cc_call_center_sk,cc_name] - CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + ColumnarToRow + InputAdapter + CometColumnarExchange [_groupingexpression,sm_type,cc_name] #1 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometProject [w_warehouse_name] [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,_groupingexpression] + CometBroadcastHashJoin [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name,d_date_sk] + CometProject [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_call_center_sk,cc_name] + CometProject [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] + CometProject [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_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] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 + CometFilter [w_warehouse_sk,w_warehouse_name] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 + CometFilter [sm_ship_mode_sk,sm_type] + CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] + CometBroadcastExchange [cc_call_center_sk,cc_name] #4 + CometFilter [cc_call_center_sk,cc_name] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + 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 1e60e40c5..4c34d3b3e 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,47 +1,44 @@ == Physical Plan == -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) +* ColumnarToRow (40) ++- CometTakeOrderedAndProject (39) + +- CometProject (38) + +- CometBroadcastHashJoin (37) + :- CometProject (33) + : +- CometBroadcastHashJoin (32) + : :- CometProject (27) + : : +- CometBroadcastHashJoin (26) + : : :- CometFilter (12) + : : : +- CometHashAggregate (11) + : : : +- CometColumnarExchange (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) + : : +- CometBroadcastExchange (25) + : : +- CometFilter (24) + : : +- CometHashAggregate (23) + : : +- CometColumnarExchange (22) + : : +- CometHashAggregate (21) + : : +- CometHashAggregate (20) + : : +- CometColumnarExchange (19) + : : +- CometHashAggregate (18) + : : +- CometProject (17) + : : +- CometBroadcastHashJoin (16) + : : :- CometFilter (14) + : : : +- CometScan parquet spark_catalog.default.store_returns (13) + : : +- ReusedExchange (15) + : +- CometBroadcastExchange (31) + : +- CometProject (30) + : +- CometFilter (29) + : +- CometScan parquet spark_catalog.default.store (28) + +- CometBroadcastExchange (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.customer (34) (1) Scan parquet spark_catalog.default.store_returns @@ -89,199 +86,179 @@ 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))] -(10) ColumnarToRow [codegen id : 1] +(10) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(12) HashAggregate [codegen id : 7] +(11) CometHashAggregate 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] -(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) +(12) CometFilter +Input [3]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11] +Condition : isnotnull(ctr_total_return#11) -(14) Scan parquet spark_catalog.default.store_returns +(13) 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 [] -PartitionFilters: [isnotnull(sr_returned_date_sk#4), dynamicpruningexpression(sr_returned_date_sk#4 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(sr_returned_date_sk#4), dynamicpruningexpression(sr_returned_date_sk#4 IN dynamicpruning#12)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(15) CometFilter +(14) 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) -(16) ReusedExchange [Reuses operator id: 6] +(15) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#6] -(17) CometBroadcastHashJoin +(16) 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 -(18) CometProject +(17) 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] -(19) CometHashAggregate +(18) 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))] -(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) CometColumnarExchange +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#13] +Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(22) HashAggregate [codegen id : 3] -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#14] +(20) CometHashAggregate +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#13] 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] - -(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#15, count#16] -Results [3]: [ctr_store_sk#11, sum#17, count#18] - -(24) Exchange -Input [3]: [ctr_store_sk#11, sum#17, count#18] -Arguments: hashpartitioning(ctr_store_sk#11, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(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)#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] - -(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) - -(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] - -(28) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ctr_store_sk#11] -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)#20) - -(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)#20, ctr_store_sk#11#21] - -(30) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#22, s_state#23] + +(21) CometHashAggregate +Input [2]: [ctr_store_sk#10, ctr_total_return#11] +Keys [1]: [ctr_store_sk#10] +Functions [1]: [partial_avg(ctr_total_return#11)] + +(22) CometColumnarExchange +Input [3]: [ctr_store_sk#10, sum#14, count#15] +Arguments: hashpartitioning(ctr_store_sk#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(23) CometHashAggregate +Input [3]: [ctr_store_sk#10, sum#14, count#15] +Keys [1]: [ctr_store_sk#10] +Functions [1]: [avg(ctr_total_return#11)] + +(24) CometFilter +Input [2]: [(avg(ctr_total_return) * 1.2)#16, ctr_store_sk#10#17] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#16) + +(25) CometBroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#16, ctr_store_sk#10#17] +Arguments: [(avg(ctr_total_return) * 1.2)#16, ctr_store_sk#10#17] + +(26) CometBroadcastHashJoin +Left output [3]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11] +Right output [2]: [(avg(ctr_total_return) * 1.2)#16, ctr_store_sk#10#17] +Arguments: [ctr_store_sk#10], [ctr_store_sk#10#17], Inner, (cast(ctr_total_return#11 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#16), BuildRight + +(27) CometProject +Input [5]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11, (avg(ctr_total_return) * 1.2)#16, ctr_store_sk#10#17] +Arguments: [ctr_customer_sk#9, ctr_store_sk#10], [ctr_customer_sk#9, ctr_store_sk#10] + +(28) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#18, s_state#19] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct -(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)) - -(32) CometProject -Input [2]: [s_store_sk#22, s_state#23] -Arguments: [s_store_sk#22], [s_store_sk#22] +(29) CometFilter +Input [2]: [s_store_sk#18, s_state#19] +Condition : ((isnotnull(s_state#19) AND (s_state#19 = TN)) AND isnotnull(s_store_sk#18)) -(33) ColumnarToRow [codegen id : 5] -Input [1]: [s_store_sk#22] +(30) CometProject +Input [2]: [s_store_sk#18, s_state#19] +Arguments: [s_store_sk#18], [s_store_sk#18] -(34) BroadcastExchange -Input [1]: [s_store_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(31) CometBroadcastExchange +Input [1]: [s_store_sk#18] +Arguments: [s_store_sk#18] -(35) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ctr_store_sk#11] -Right keys [1]: [s_store_sk#22] -Join type: Inner -Join condition: None +(32) CometBroadcastHashJoin +Left output [2]: [ctr_customer_sk#9, ctr_store_sk#10] +Right output [1]: [s_store_sk#18] +Arguments: [ctr_store_sk#10], [s_store_sk#18], Inner, BuildRight -(36) Project [codegen id : 7] -Output [1]: [ctr_customer_sk#10] -Input [3]: [ctr_customer_sk#10, ctr_store_sk#11, s_store_sk#22] +(33) CometProject +Input [3]: [ctr_customer_sk#9, ctr_store_sk#10, s_store_sk#18] +Arguments: [ctr_customer_sk#9], [ctr_customer_sk#9] -(37) Scan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#24, c_customer_id#25] +(34) Scan parquet spark_catalog.default.customer +Output [2]: [c_customer_sk#20, c_customer_id#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(38) CometFilter -Input [2]: [c_customer_sk#24, c_customer_id#25] -Condition : isnotnull(c_customer_sk#24) +(35) CometFilter +Input [2]: [c_customer_sk#20, c_customer_id#21] +Condition : isnotnull(c_customer_sk#20) -(39) ColumnarToRow [codegen id : 6] -Input [2]: [c_customer_sk#24, c_customer_id#25] +(36) CometBroadcastExchange +Input [2]: [c_customer_sk#20, c_customer_id#21] +Arguments: [c_customer_sk#20, c_customer_id#21] -(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] +(37) CometBroadcastHashJoin +Left output [1]: [ctr_customer_sk#9] +Right output [2]: [c_customer_sk#20, c_customer_id#21] +Arguments: [ctr_customer_sk#9], [c_customer_sk#20], Inner, BuildRight -(41) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ctr_customer_sk#10] -Right keys [1]: [c_customer_sk#24] -Join type: Inner -Join condition: None +(38) CometProject +Input [3]: [ctr_customer_sk#9, c_customer_sk#20, c_customer_id#21] +Arguments: [c_customer_id#21], [c_customer_id#21] -(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] +(39) CometTakeOrderedAndProject +Input [1]: [c_customer_id#21] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#21 ASC NULLS FIRST], output=[c_customer_id#21]), [c_customer_id#21], 100, [c_customer_id#21 ASC NULLS FIRST], [c_customer_id#21] -(43) TakeOrderedAndProject -Input [1]: [c_customer_id#25] -Arguments: 100, [c_customer_id#25 ASC NULLS FIRST], [c_customer_id#25] +(40) ColumnarToRow [codegen id : 1] +Input [1]: [c_customer_id#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (48) -+- * ColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan parquet spark_catalog.default.date_dim (44) +BroadcastExchange (45) ++- * ColumnarToRow (44) + +- CometProject (43) + +- CometFilter (42) + +- CometScan parquet spark_catalog.default.date_dim (41) -(44) Scan parquet spark_catalog.default.date_dim +(41) 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 -(45) CometFilter +(42) 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)) -(46) CometProject +(43) CometProject Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(47) ColumnarToRow [codegen id : 1] +(44) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(48) BroadcastExchange +(45) BroadcastExchange Input [1]: [d_date_sk#6] -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=4] -Subquery:2 Hosting operator id = 14 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 13 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 238f9b702..eb1ae9719 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,69 +1,51 @@ -TakeOrderedAndProject [c_customer_id] - WholeStageCodegen (7) - Project [c_customer_id] - BroadcastHashJoin [ctr_customer_sk,c_customer_sk] - Project [ctr_customer_sk] - BroadcastHashJoin [ctr_store_sk,s_store_sk] - Project [ctr_customer_sk,ctr_store_sk] - BroadcastHashJoin [ctr_store_sk,ctr_store_sk,ctr_total_return,(avg(ctr_total_return) * 1.2)] - Filter [ctr_total_return] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [sr_customer_sk,sr_store_sk,sum,sr_return_amt] - CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] - CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - 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] #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] #6 - WholeStageCodegen (2) +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [c_customer_id] + CometProject [c_customer_id] + CometBroadcastHashJoin [ctr_customer_sk,c_customer_sk,c_customer_id] + CometProject [ctr_customer_sk] + CometBroadcastHashJoin [ctr_customer_sk,ctr_store_sk,s_store_sk] + CometProject [ctr_customer_sk,ctr_store_sk] + CometBroadcastHashJoin [ctr_customer_sk,ctr_store_sk,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_store_sk] + CometFilter [ctr_customer_sk,ctr_store_sk,ctr_total_return] + CometHashAggregate [ctr_customer_sk,ctr_store_sk,ctr_total_return,sr_customer_sk,sr_store_sk,sum,sum(UnscaledValue(sr_return_amt))] + CometColumnarExchange [sr_customer_sk,sr_store_sk] #1 + CometHashAggregate [sr_customer_sk,sr_store_sk,sum,sr_return_amt] + CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] + CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] + CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [sr_customer_sk,sr_store_sk,sum,sr_return_amt] - CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] - CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_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 - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id] + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_store_sk] #4 + CometFilter [(avg(ctr_total_return) * 1.2),ctr_store_sk] + CometHashAggregate [(avg(ctr_total_return) * 1.2),ctr_store_sk,ctr_store_sk,sum,count,avg(ctr_total_return)] + CometColumnarExchange [ctr_store_sk] #5 + CometHashAggregate [ctr_store_sk,sum,count,ctr_total_return] + CometHashAggregate [ctr_store_sk,ctr_total_return,sr_customer_sk,sr_store_sk,sum,sum(UnscaledValue(sr_return_amt))] + CometColumnarExchange [sr_customer_sk,sr_store_sk] #6 + CometHashAggregate [sr_customer_sk,sr_store_sk,sum,sr_return_amt] + CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] + CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] + CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_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 + ReusedExchange [d_date_sk] #3 + CometBroadcastExchange [s_store_sk] #7 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometBroadcastExchange [c_customer_sk,c_customer_id] #8 + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id] 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 4a29b7260..6f8c539fb 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,49 +1,51 @@ == Physical Plan == -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) +TakeOrderedAndProject (47) ++- * HashAggregate (46) + +- * ColumnarToRow (45) + +- CometColumnarExchange (44) + +- RowToColumnar (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 @@ -243,50 +245,56 @@ Functions [1]: [partial_count(1)] 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] -(43) Exchange +(43) RowToColumnar 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] -(44) HashAggregate [codegen id : 6] +(44) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=5] + +(45) ColumnarToRow [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] + +(46) 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)#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] -(45) TakeOrderedAndProject +(47) 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 = 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) +BroadcastExchange (52) ++- * ColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) -(46) Scan parquet spark_catalog.default.date_dim +(48) 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 -(47) CometFilter +(49) 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)) -(48) CometProject +(50) CometProject Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(49) ColumnarToRow [codegen id : 1] +(51) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(50) BroadcastExchange +(52) BroadcastExchange Input [1]: [d_date_sk#9] 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/q10/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt index efd4b187d..a3ab6bbbb 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,71 +1,73 @@ 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 (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 (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] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] - CometFilter [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] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,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 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometColumnarExchange [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 + RowToColumnar + 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] + Project [c_current_cdemo_sk] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] + Filter [exists,exists] + BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,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 + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [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] + CometBroadcastExchange [ss_customer_sk] #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,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 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [cs_ship_customer_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,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 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_county] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) + BroadcastExchange #8 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [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] - 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] + CometFilter [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] + 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-v1_4/q11/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt index 1c5d5222e..ac0d82f80 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 @@ -1,76 +1,73 @@ == Physical Plan == -TakeOrderedAndProject (72) -+- * Project (71) - +- * BroadcastHashJoin Inner BuildRight (70) - :- * 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) - +- * 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) +* ColumnarToRow (69) ++- CometTakeOrderedAndProject (68) + +- CometProject (67) + +- CometBroadcastHashJoin (66) + :- CometProject (51) + : +- CometBroadcastHashJoin (50) + : :- CometProject (34) + : : +- CometBroadcastHashJoin (33) + : : :- CometFilter (16) + : : : +- CometHashAggregate (15) + : : : +- CometColumnarExchange (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) + : : +- CometBroadcastExchange (32) + : : +- CometHashAggregate (31) + : : +- CometColumnarExchange (30) + : : +- CometHashAggregate (29) + : : +- CometProject (28) + : : +- CometBroadcastHashJoin (27) + : : :- CometProject (23) + : : : +- CometBroadcastHashJoin (22) + : : : :- CometFilter (18) + : : : : +- CometScan parquet spark_catalog.default.customer (17) + : : : +- CometBroadcastExchange (21) + : : : +- CometFilter (20) + : : : +- CometScan parquet spark_catalog.default.store_sales (19) + : : +- CometBroadcastExchange (26) + : : +- CometFilter (25) + : : +- CometScan parquet spark_catalog.default.date_dim (24) + : +- CometBroadcastExchange (49) + : +- CometFilter (48) + : +- CometHashAggregate (47) + : +- CometColumnarExchange (46) + : +- CometHashAggregate (45) + : +- CometProject (44) + : +- CometBroadcastHashJoin (43) + : :- CometProject (41) + : : +- CometBroadcastHashJoin (40) + : : :- CometFilter (36) + : : : +- CometScan parquet spark_catalog.default.customer (35) + : : +- CometBroadcastExchange (39) + : : +- CometFilter (38) + : : +- CometScan parquet spark_catalog.default.web_sales (37) + : +- ReusedExchange (42) + +- CometBroadcastExchange (65) + +- CometHashAggregate (64) + +- CometColumnarExchange (63) + +- CometHashAggregate (62) + +- CometProject (61) + +- CometBroadcastHashJoin (60) + :- CometProject (58) + : +- CometBroadcastHashJoin (57) + : :- CometFilter (53) + : : +- CometScan parquet spark_catalog.default.customer (52) + : +- CometBroadcastExchange (56) + : +- CometFilter (55) + : +- CometScan parquet spark_catalog.default.web_sales (54) + +- ReusedExchange (59) (1) Scan parquet spark_catalog.default.customer @@ -138,341 +135,321 @@ Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_fl 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)))] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(16) HashAggregate [codegen id : 8] +(15) CometHashAggregate 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)))#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] -(17) Filter [codegen id : 8] -Input [2]: [customer_id#18, year_total#19] -Condition : (isnotnull(year_total#19) AND (year_total#19 > 0.00)) +(16) CometFilter +Input [2]: [customer_id#17, year_total#18] +Condition : (isnotnull(year_total#18) AND (year_total#18 > 0.00)) -(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] +(17) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(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)) +(18) CometFilter +Input [8]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26] +Condition : (isnotnull(c_customer_sk#19) AND isnotnull(c_customer_id#20)) -(20) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] +(19) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_customer_sk#27, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#31), dynamicpruningexpression(ss_sold_date_sk#31 IN dynamicpruning#32)] +PartitionFilters: [isnotnull(ss_sold_date_sk#30), dynamicpruningexpression(ss_sold_date_sk#30 IN dynamicpruning#31)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(21) CometFilter -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) +(20) CometFilter +Input [4]: [ss_customer_sk#27, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30] +Condition : isnotnull(ss_customer_sk#27) -(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] +(21) CometBroadcastExchange +Input [4]: [ss_customer_sk#27, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30] +Arguments: [ss_customer_sk#27, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30] -(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 +(22) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26] +Right output [4]: [ss_customer_sk#27, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30] +Arguments: [c_customer_sk#19], [ss_customer_sk#27], Inner, BuildRight -(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] +(23) CometProject +Input [12]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, ss_customer_sk#27, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30] +Arguments: [c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30], [c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30] -(25) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_year#34] +(24) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#32, d_year#33] 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 -(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 - -(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] - -(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] +(25) CometFilter +Input [2]: [d_date_sk#32, d_year#33] +Condition : ((isnotnull(d_year#33) AND (d_year#33 = 2002)) AND isnotnull(d_date_sk#32)) + +(26) CometBroadcastExchange +Input [2]: [d_date_sk#32, d_year#33] +Arguments: [d_date_sk#32, d_year#33] + +(27) CometBroadcastHashJoin +Left output [10]: [c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30] +Right output [2]: [d_date_sk#32, d_year#33] +Arguments: [ss_sold_date_sk#30], [d_date_sk#32], Inner, BuildRight + +(28) CometProject +Input [12]: [c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30, d_date_sk#32, d_year#33] +Arguments: [c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, ss_ext_discount_amt#28, ss_ext_list_price#29, d_year#33], [c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, ss_ext_discount_amt#28, ss_ext_list_price#29, d_year#33] + +(29) CometHashAggregate +Input [10]: [c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, ss_ext_discount_amt#28, ss_ext_list_price#29, d_year#33] +Keys [8]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#33, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26] +Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#29 - ss_ext_discount_amt#28)))] + +(30) CometColumnarExchange +Input [9]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#33, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, sum#34] +Arguments: hashpartitioning(c_customer_id#20, c_first_name#21, c_last_name#22, d_year#33, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(31) CometHashAggregate +Input [9]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#33, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, sum#34] +Keys [8]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#33, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26] +Functions [1]: [sum(UnscaledValue((ss_ext_list_price#29 - ss_ext_discount_amt#28)))] + +(32) CometBroadcastExchange +Input [3]: [customer_id#35, customer_preferred_cust_flag#36, year_total#37] +Arguments: [customer_id#35, customer_preferred_cust_flag#36, year_total#37] + +(33) CometBroadcastHashJoin +Left output [2]: [customer_id#17, year_total#18] +Right output [3]: [customer_id#35, customer_preferred_cust_flag#36, year_total#37] +Arguments: [customer_id#17], [customer_id#35], Inner, BuildRight + +(34) CometProject +Input [5]: [customer_id#17, year_total#18, customer_id#35, customer_preferred_cust_flag#36, year_total#37] +Arguments: [customer_id#17, year_total#18, customer_preferred_cust_flag#36, year_total#37], [customer_id#17, year_total#18, customer_preferred_cust_flag#36, year_total#37] + +(35) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#38, c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#43, c_login#44, c_email_address#45] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(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)) +(36) CometFilter +Input [8]: [c_customer_sk#38, c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#43, c_login#44, c_email_address#45] +Condition : (isnotnull(c_customer_sk#38) AND isnotnull(c_customer_id#39)) -(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] +(37) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_bill_customer_sk#46, ws_ext_discount_amt#47, ws_ext_list_price#48, ws_sold_date_sk#49] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#50), dynamicpruningexpression(ws_sold_date_sk#50 IN dynamicpruning#51)] +PartitionFilters: [isnotnull(ws_sold_date_sk#49), dynamicpruningexpression(ws_sold_date_sk#49 IN dynamicpruning#50)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(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) +(38) CometFilter +Input [4]: [ws_bill_customer_sk#46, ws_ext_discount_amt#47, ws_ext_list_price#48, ws_sold_date_sk#49] +Condition : isnotnull(ws_bill_customer_sk#46) + +(39) CometBroadcastExchange +Input [4]: [ws_bill_customer_sk#46, ws_ext_discount_amt#47, ws_ext_list_price#48, ws_sold_date_sk#49] +Arguments: [ws_bill_customer_sk#46, ws_ext_discount_amt#47, ws_ext_list_price#48, ws_sold_date_sk#49] + +(40) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#38, c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#43, c_login#44, c_email_address#45] +Right output [4]: [ws_bill_customer_sk#46, ws_ext_discount_amt#47, ws_ext_list_price#48, ws_sold_date_sk#49] +Arguments: [c_customer_sk#38], [ws_bill_customer_sk#46], Inner, BuildRight -(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) CometProject +Input [12]: [c_customer_sk#38, c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#43, c_login#44, c_email_address#45, ws_bill_customer_sk#46, ws_ext_discount_amt#47, ws_ext_list_price#48, ws_sold_date_sk#49] +Arguments: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#43, c_login#44, c_email_address#45, ws_ext_discount_amt#47, ws_ext_list_price#48, ws_sold_date_sk#49], [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#43, c_login#44, c_email_address#45, ws_ext_discount_amt#47, ws_ext_list_price#48, ws_sold_date_sk#49] -(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) ReusedExchange [Reuses operator id: 10] +Output [2]: [d_date_sk#51, d_year#52] -(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) CometBroadcastHashJoin +Left output [10]: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#43, c_login#44, c_email_address#45, ws_ext_discount_amt#47, ws_ext_list_price#48, ws_sold_date_sk#49] +Right output [2]: [d_date_sk#51, d_year#52] +Arguments: [ws_sold_date_sk#49], [d_date_sk#51], Inner, BuildRight -(44) ReusedExchange [Reuses operator id: 10] -Output [2]: [d_date_sk#52, d_year#53] +(44) CometProject +Input [12]: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#43, c_login#44, c_email_address#45, ws_ext_discount_amt#47, ws_ext_list_price#48, ws_sold_date_sk#49, d_date_sk#51, d_year#52] +Arguments: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#43, c_login#44, c_email_address#45, ws_ext_discount_amt#47, ws_ext_list_price#48, d_year#52], [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#43, c_login#44, c_email_address#45, ws_ext_discount_amt#47, ws_ext_list_price#48, d_year#52] -(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) CometHashAggregate +Input [10]: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#43, c_login#44, c_email_address#45, ws_ext_discount_amt#47, ws_ext_list_price#48, d_year#52] +Keys [8]: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#43, c_login#44, c_email_address#45, d_year#52] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#48 - ws_ext_discount_amt#47)))] -(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] +(46) CometColumnarExchange +Input [9]: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#43, c_login#44, c_email_address#45, d_year#52, sum#53] +Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#43, c_login#44, c_email_address#45, d_year#52, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (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] - -(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 - -(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] - -(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] +Input [9]: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#43, c_login#44, c_email_address#45, d_year#52, sum#53] +Keys [8]: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#43, c_login#44, c_email_address#45, d_year#52] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#48 - ws_ext_discount_amt#47)))] + +(48) CometFilter +Input [2]: [customer_id#54, year_total#55] +Condition : (isnotnull(year_total#55) AND (year_total#55 > 0.00)) + +(49) CometBroadcastExchange +Input [2]: [customer_id#54, year_total#55] +Arguments: [customer_id#54, year_total#55] + +(50) CometBroadcastHashJoin +Left output [4]: [customer_id#17, year_total#18, customer_preferred_cust_flag#36, year_total#37] +Right output [2]: [customer_id#54, year_total#55] +Arguments: [customer_id#17], [customer_id#54], Inner, BuildRight + +(51) CometProject +Input [6]: [customer_id#17, year_total#18, customer_preferred_cust_flag#36, year_total#37, customer_id#54, year_total#55] +Arguments: [customer_id#17, year_total#18, customer_preferred_cust_flag#36, year_total#37, year_total#55], [customer_id#17, year_total#18, customer_preferred_cust_flag#36, year_total#37, year_total#55] + +(52) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#56, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(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)) +(53) CometFilter +Input [8]: [c_customer_sk#56, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63] +Condition : (isnotnull(c_customer_sk#56) AND isnotnull(c_customer_id#57)) -(57) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] +(54) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#69), dynamicpruningexpression(ws_sold_date_sk#69 IN dynamicpruning#70)] +PartitionFilters: [isnotnull(ws_sold_date_sk#67), dynamicpruningexpression(ws_sold_date_sk#67 IN dynamicpruning#68)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(58) CometFilter -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) +(55) CometFilter +Input [4]: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] +Condition : isnotnull(ws_bill_customer_sk#64) + +(56) CometBroadcastExchange +Input [4]: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] +Arguments: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] + +(57) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#56, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63] +Right output [4]: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] +Arguments: [c_customer_sk#56], [ws_bill_customer_sk#64], Inner, BuildRight + +(58) CometProject +Input [12]: [c_customer_sk#56, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63, ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] +Arguments: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67], [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] -(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] +(59) ReusedExchange [Reuses operator id: 26] +Output [2]: [d_date_sk#69, d_year#70] (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 +Left output [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] +Right output [2]: [d_date_sk#69, d_year#70] +Arguments: [ws_sold_date_sk#67], [d_date_sk#69], Inner, BuildRight (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) ReusedExchange [Reuses operator id: 27] -Output [2]: [d_date_sk#71, d_year#72] - -(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) 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) 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) 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#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 : 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#74, year_total#75] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] - -(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#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 : 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#37] -Arguments: 100, [customer_preferred_cust_flag#37 ASC NULLS FIRST], [customer_preferred_cust_flag#37] +Input [12]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67, d_date_sk#69, d_year#70] +Arguments: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, d_year#70], [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, d_year#70] + +(62) CometHashAggregate +Input [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, d_year#70] +Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63, d_year#70] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#66 - ws_ext_discount_amt#65)))] + +(63) CometColumnarExchange +Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63, d_year#70, sum#71] +Arguments: hashpartitioning(c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63, d_year#70, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(64) CometHashAggregate +Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63, d_year#70, sum#71] +Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63, d_year#70] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#66 - ws_ext_discount_amt#65)))] + +(65) CometBroadcastExchange +Input [2]: [customer_id#72, year_total#73] +Arguments: [customer_id#72, year_total#73] + +(66) CometBroadcastHashJoin +Left output [5]: [customer_id#17, year_total#18, customer_preferred_cust_flag#36, year_total#37, year_total#55] +Right output [2]: [customer_id#72, year_total#73] +Arguments: [customer_id#17], [customer_id#72], Inner, (CASE WHEN (year_total#55 > 0.00) THEN (year_total#73 / year_total#55) END > CASE WHEN (year_total#18 > 0.00) THEN (year_total#37 / year_total#18) END), BuildRight + +(67) CometProject +Input [7]: [customer_id#17, year_total#18, customer_preferred_cust_flag#36, year_total#37, year_total#55, customer_id#72, year_total#73] +Arguments: [customer_preferred_cust_flag#36], [customer_preferred_cust_flag#36] + +(68) CometTakeOrderedAndProject +Input [1]: [customer_preferred_cust_flag#36] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_preferred_cust_flag#36 ASC NULLS FIRST], output=[customer_preferred_cust_flag#36]), [customer_preferred_cust_flag#36], 100, [customer_preferred_cust_flag#36 ASC NULLS FIRST], [customer_preferred_cust_flag#36] + +(69) ColumnarToRow [codegen id : 1] +Input [1]: [customer_preferred_cust_flag#36] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (76) -+- * ColumnarToRow (75) - +- CometFilter (74) - +- CometScan parquet spark_catalog.default.date_dim (73) +BroadcastExchange (73) ++- * ColumnarToRow (72) + +- CometFilter (71) + +- CometScan parquet spark_catalog.default.date_dim (70) -(73) Scan parquet spark_catalog.default.date_dim +(70) 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 -(74) CometFilter +(71) 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)) -(75) ColumnarToRow [codegen id : 1] +(72) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#14, d_year#15] -(76) BroadcastExchange +(73) BroadcastExchange Input [2]: [d_date_sk#14, d_year#15] -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=5] -Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#31 IN dynamicpruning#32 -BroadcastExchange (80) -+- * ColumnarToRow (79) - +- CometFilter (78) - +- CometScan parquet spark_catalog.default.date_dim (77) +Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#31 +BroadcastExchange (77) ++- * ColumnarToRow (76) + +- CometFilter (75) + +- CometScan parquet spark_catalog.default.date_dim (74) -(77) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_year#34] +(74) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#32, d_year#33] 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#33, d_year#34] -Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2002)) AND isnotnull(d_date_sk#33)) +(75) CometFilter +Input [2]: [d_date_sk#32, d_year#33] +Condition : ((isnotnull(d_year#33) AND (d_year#33 = 2002)) AND isnotnull(d_date_sk#32)) -(79) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#33, d_year#34] +(76) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#32, d_year#33] -(80) BroadcastExchange -Input [2]: [d_date_sk#33, d_year#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] +(77) BroadcastExchange +Input [2]: [d_date_sk#32, d_year#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#50 IN dynamicpruning#13 +Subquery:3 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#49 IN dynamicpruning#13 -Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#69 IN dynamicpruning#32 +Subquery:4 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#67 IN dynamicpruning#31 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 127a5a291..83ffbd0a6 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,107 +1,87 @@ -TakeOrderedAndProject [customer_preferred_cust_flag] - 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] - BroadcastHashJoin [customer_id,customer_id] - Project [customer_id,year_total,customer_preferred_cust_flag,year_total] - BroadcastHashJoin [customer_id,customer_id] - Filter [year_total] - 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 (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,sum,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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_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_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - 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,sum,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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 - CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_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 #8 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - 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] #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,sum,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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 - CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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 #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] #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,sum,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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [customer_preferred_cust_flag] + CometProject [customer_preferred_cust_flag] + CometBroadcastHashJoin [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total,customer_id,year_total] + CometProject [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total] + CometBroadcastHashJoin [customer_id,year_total,customer_preferred_cust_flag,year_total,customer_id,year_total] + CometProject [customer_id,year_total,customer_preferred_cust_flag,year_total] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_preferred_cust_flag,year_total] + CometFilter [customer_id,year_total] + CometHashAggregate [customer_id,year_total,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)))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 - CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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 + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_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_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [customer_id,customer_preferred_cust_flag,year_total] #5 + CometHashAggregate [customer_id,customer_preferred_cust_flag,year_total,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)))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #6 + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + 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 [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_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 #8 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [customer_id,year_total] #10 + CometFilter [customer_id,year_total] + CometHashAggregate [customer_id,year_total,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)))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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 + CometBroadcastExchange [customer_id,year_total] #13 + CometHashAggregate [customer_id,year_total,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)))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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 160f0c593..40c6660a9 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 @@ -2,11 +2,11 @@ TakeOrderedAndProject (22) +- * Project (21) +- Window (20) - +- * Sort (19) - +- Exchange (18) - +- * HashAggregate (17) - +- Exchange (16) - +- * ColumnarToRow (15) + +- * ColumnarToRow (19) + +- CometSort (18) + +- CometColumnarExchange (17) + +- CometHashAggregate (16) + +- CometColumnarExchange (15) +- CometHashAggregate (14) +- CometProject (13) +- CometBroadcastHashJoin (12) @@ -92,39 +92,37 @@ Input [6]: [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, 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))] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(16) CometHashAggregate 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] -(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=2] +(17) CometColumnarExchange +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(18) CometSort +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] +Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6], [i_class#9 ASC NULLS FIRST] -(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 +(19) ColumnarToRow [codegen id : 1] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] (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] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] +Arguments: [sum(_w0#15) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#16], [i_class#9] -(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] +(21) Project [codegen id : 2] +Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, ((_w0#15 * 100) / _we0#16) AS revenueratio#17, i_item_id#6] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6, _we0#16] (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] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17, 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#17 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] ===== Subqueries ===== 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 4c2d3f3d7..3e0fd16a4 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,38 +1,34 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (4) + WholeStageCodegen (2) Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] InputAdapter Window [_w0,i_class] - WholeStageCodegen (3) - Sort [i_class] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [i_class] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ws_ext_sales_price] - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometColumnarExchange [i_class] #1 + CometHashAggregate [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ws_ext_sales_price] + CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + 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 1cda12261..b05df2927 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,7 +1,7 @@ == Physical Plan == -* HashAggregate (33) -+- Exchange (32) - +- * ColumnarToRow (31) +* ColumnarToRow (33) ++- CometHashAggregate (32) + +- CometColumnarExchange (31) +- CometHashAggregate (30) +- CometProject (29) +- CometBroadcastHashJoin (28) @@ -179,19 +179,17 @@ 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))] -(31) ColumnarToRow [codegen id : 1] +(31) CometColumnarExchange Input [7]: [sum#23, count#24, sum#25, count#26, sum#27, count#28, sum#29] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(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] +(32) CometHashAggregate 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)#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] + +(33) ColumnarToRow [codegen id : 1] +Input [4]: [avg(ss_quantity)#30, avg(ss_ext_sales_price)#31, avg(ss_ext_wholesale_cost)#32, sum(ss_ext_wholesale_cost)#33] ===== Subqueries ===== 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 98d995db0..0f438f153 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,45 +1,43 @@ -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] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [sum,count,sum,count,sum,count,sum,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,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status,hd_demo_sk,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,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_demo_sk,cd_marital_status,cd_education_status] - CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost] - CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,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_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,ca_address_sk,ca_state] - 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_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,s_store_sk] - CometFilter [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] - 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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk] #3 - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange [ca_address_sk,ca_state] #4 - CometProject [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #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 [hd_demo_sk,hd_dep_count] #7 - CometFilter [hd_demo_sk,hd_dep_count] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] + CometHashAggregate [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,avg(ss_quantity),avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),sum(UnscaledValue(ss_ext_wholesale_cost))] + CometColumnarExchange #1 + CometHashAggregate [sum,count,sum,count,sum,count,sum,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,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status,hd_demo_sk,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,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_demo_sk,cd_marital_status,cd_education_status] + CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,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_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,ca_address_sk,ca_state] + 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_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,s_store_sk] + CometFilter [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] + 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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [s_store_sk] #3 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] + CometBroadcastExchange [ca_address_sk,ca_state] #4 + CometProject [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #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 [hd_demo_sk,hd_dep_count] #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 0b4712fc0..6789d55ce 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 @@ -1,109 +1,106 @@ == Physical Plan == -TakeOrderedAndProject (105) -+- * HashAggregate (104) - +- Exchange (103) - +- * HashAggregate (102) - +- * Expand (101) - +- Union (100) - :- * Project (67) - : +- * Filter (66) - : +- * HashAggregate (65) - : +- Exchange (64) - : +- * HashAggregate (63) - : +- * Project (62) - : +- * BroadcastHashJoin Inner BuildRight (61) - : :- * Project (59) - : : +- * BroadcastHashJoin Inner BuildRight (58) - : : :- * BroadcastHashJoin LeftSemi BuildRight (51) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (50) - : : : +- * Project (49) - : : : +- * BroadcastHashJoin Inner BuildRight (48) - : : : :- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.item (4) - : : : +- BroadcastExchange (47) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) - : : : :- * HashAggregate (35) - : : : : +- Exchange (34) - : : : : +- * 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) - : : : +- * 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) - : : : +- CometFilter (53) - : : : +- CometScan parquet spark_catalog.default.item (52) - : : +- ReusedExchange (55) - : +- ReusedExchange (60) - :- * Project (83) - : +- * Filter (82) - : +- * HashAggregate (81) - : +- Exchange (80) - : +- * HashAggregate (79) - : +- * Project (78) - : +- * BroadcastHashJoin Inner BuildRight (77) - : :- * Project (75) - : : +- * BroadcastHashJoin Inner BuildRight (74) - : : :- * BroadcastHashJoin LeftSemi BuildRight (72) - : : : :- * ColumnarToRow (70) - : : : : +- CometFilter (69) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (68) - : : : +- ReusedExchange (71) - : : +- ReusedExchange (73) - : +- ReusedExchange (76) - +- * Project (99) - +- * Filter (98) - +- * HashAggregate (97) - +- Exchange (96) - +- * HashAggregate (95) - +- * Project (94) - +- * BroadcastHashJoin Inner BuildRight (93) - :- * Project (91) - : +- * BroadcastHashJoin Inner BuildRight (90) - : :- * BroadcastHashJoin LeftSemi BuildRight (88) - : : :- * ColumnarToRow (86) - : : : +- CometFilter (85) - : : : +- CometScan parquet spark_catalog.default.web_sales (84) - : : +- ReusedExchange (87) - : +- ReusedExchange (89) - +- ReusedExchange (92) +* ColumnarToRow (102) ++- CometTakeOrderedAndProject (101) + +- CometHashAggregate (100) + +- CometColumnarExchange (99) + +- CometHashAggregate (98) + +- CometExpand (97) + +- CometUnion (96) + :- CometProject (65) + : +- CometFilter (64) + : +- CometHashAggregate (63) + : +- CometColumnarExchange (62) + : +- CometHashAggregate (61) + : +- CometProject (60) + : +- CometBroadcastHashJoin (59) + : :- CometProject (54) + : : +- CometBroadcastHashJoin (53) + : : :- CometBroadcastHashJoin (47) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (46) + : : : +- CometProject (45) + : : : +- CometBroadcastHashJoin (44) + : : : :- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.item (3) + : : : +- CometBroadcastExchange (43) + : : : +- CometBroadcastHashJoin (42) + : : : :- CometHashAggregate (32) + : : : : +- CometColumnarExchange (31) + : : : : +- CometHashAggregate (30) + : : : : +- CometProject (29) + : : : : +- CometBroadcastHashJoin (28) + : : : : :- CometProject (26) + : : : : : +- CometBroadcastHashJoin (25) + : : : : : :- CometFilter (6) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (5) + : : : : : +- CometBroadcastExchange (24) + : : : : : +- CometBroadcastHashJoin (23) + : : : : : :- CometFilter (8) + : : : : : : +- CometScan parquet spark_catalog.default.item (7) + : : : : : +- CometBroadcastExchange (22) + : : : : : +- CometProject (21) + : : : : : +- CometBroadcastHashJoin (20) + : : : : : :- CometProject (15) + : : : : : : +- CometBroadcastHashJoin (14) + : : : : : : :- CometFilter (10) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (9) + : : : : : : +- CometBroadcastExchange (13) + : : : : : : +- CometFilter (12) + : : : : : : +- CometScan parquet spark_catalog.default.item (11) + : : : : : +- CometBroadcastExchange (19) + : : : : : +- CometProject (18) + : : : : : +- CometFilter (17) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (16) + : : : : +- ReusedExchange (27) + : : : +- CometBroadcastExchange (41) + : : : +- CometProject (40) + : : : +- CometBroadcastHashJoin (39) + : : : :- CometProject (37) + : : : : +- CometBroadcastHashJoin (36) + : : : : :- CometFilter (34) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (33) + : : : : +- ReusedExchange (35) + : : : +- ReusedExchange (38) + : : +- CometBroadcastExchange (52) + : : +- CometBroadcastHashJoin (51) + : : :- CometFilter (49) + : : : +- CometScan parquet spark_catalog.default.item (48) + : : +- ReusedExchange (50) + : +- CometBroadcastExchange (58) + : +- CometProject (57) + : +- CometFilter (56) + : +- CometScan parquet spark_catalog.default.date_dim (55) + :- CometProject (80) + : +- CometFilter (79) + : +- CometHashAggregate (78) + : +- CometColumnarExchange (77) + : +- CometHashAggregate (76) + : +- CometProject (75) + : +- CometBroadcastHashJoin (74) + : :- CometProject (72) + : : +- CometBroadcastHashJoin (71) + : : :- CometBroadcastHashJoin (69) + : : : :- CometFilter (67) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (66) + : : : +- ReusedExchange (68) + : : +- ReusedExchange (70) + : +- ReusedExchange (73) + +- CometProject (95) + +- CometFilter (94) + +- CometHashAggregate (93) + +- CometColumnarExchange (92) + +- CometHashAggregate (91) + +- CometProject (90) + +- CometBroadcastHashJoin (89) + :- CometProject (87) + : +- CometBroadcastHashJoin (86) + : :- CometBroadcastHashJoin (84) + : : :- CometFilter (82) + : : : +- CometScan parquet spark_catalog.default.web_sales (81) + : : +- ReusedExchange (83) + : +- ReusedExchange (85) + +- ReusedExchange (88) (1) Scan parquet spark_catalog.default.store_sales @@ -118,24 +115,18 @@ 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 : 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 +(3) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) -(6) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(7) Scan parquet spark_catalog.default.store_sales +(5) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] @@ -143,22 +134,22 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(8) CometFilter +(6) CometFilter Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(9) Scan parquet spark_catalog.default.item +(7) 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 -(10) CometFilter +(8) 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)) -(11) Scan parquet spark_catalog.default.catalog_sales +(9) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] @@ -166,116 +157,111 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(12) CometFilter +(10) CometFilter Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) -(13) Scan parquet spark_catalog.default.item +(11) 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 -(14) CometFilter +(12) 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) -(15) CometBroadcastExchange +(13) 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] -(16) CometBroadcastHashJoin +(14) 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 -(17) CometProject +(15) 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 +(16) 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 -(19) CometFilter +(17) 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 +(18) CometProject Input [2]: [d_date_sk#24, d_year#25] Arguments: [d_date_sk#24], [d_date_sk#24] -(21) CometBroadcastExchange +(19) CometBroadcastExchange Input [1]: [d_date_sk#24] Arguments: [d_date_sk#24] -(22) CometBroadcastHashJoin +(20) 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 +(21) 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] -(24) CometBroadcastExchange +(22) CometBroadcastExchange Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] -(25) CometBroadcastHashJoin +(23) 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 -(26) CometBroadcastExchange +(24) CometBroadcastExchange Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(27) CometBroadcastHashJoin +(25) 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 -(28) CometProject +(26) 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] -(29) ReusedExchange [Reuses operator id: 21] +(27) ReusedExchange [Reuses operator id: 19] Output [1]: [d_date_sk#26] -(30) CometBroadcastHashJoin +(28) 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 -(31) CometProject +(29) 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] -(32) CometHashAggregate +(30) CometHashAggregate Input [3]: [brand_id#27, class_id#28, category_id#29] Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] -(33) ColumnarToRow [codegen id : 1] -Input [3]: [brand_id#27, class_id#28, category_id#29] - -(34) Exchange +(31) CometColumnarExchange 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] +Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(35) HashAggregate [codegen id : 3] +(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#27, class_id#28, category_id#29] -(36) Scan parquet spark_catalog.default.web_sales +(33) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] @@ -283,506 +269,486 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(37) CometFilter +(34) CometFilter Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] Condition : isnotnull(ws_item_sk#30) -(38) ReusedExchange [Reuses operator id: 15] +(35) ReusedExchange [Reuses operator id: 13] Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -(39) CometBroadcastHashJoin +(36) 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) CometProject +(37) 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) ReusedExchange [Reuses operator id: 21] +(38) ReusedExchange [Reuses operator id: 19] Output [1]: [d_date_sk#37] -(42) CometBroadcastHashJoin +(39) 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) CometProject +(40) 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) ColumnarToRow [codegen id : 2] +(41) CometBroadcastExchange Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] -(45) BroadcastExchange -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 : 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 +(42) CometBroadcastHashJoin +Left output [3]: [brand_id#27, class_id#28, category_id#29] +Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [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)], [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)], LeftSemi, BuildRight -(47) BroadcastExchange +(43) CometBroadcastExchange 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] +Arguments: [brand_id#27, class_id#28, category_id#29] -(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#27, class_id#28, category_id#29] -Join type: Inner -Join condition: None +(44) CometBroadcastHashJoin +Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Right output [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight -(49) Project [codegen id : 4] -Output [1]: [i_item_sk#6 AS ss_item_sk#38] +(45) CometProject 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] +Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] -(50) BroadcastExchange +(46) CometBroadcastExchange Input [1]: [ss_item_sk#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: [ss_item_sk#38] -(51) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#38] -Join type: LeftSemi -Join condition: None +(47) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Right output [1]: [ss_item_sk#38] +Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight -(52) Scan parquet spark_catalog.default.item +(48) Scan parquet spark_catalog.default.item 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 +(49) CometFilter 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 : 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] +(50) ReusedExchange [Reuses operator id: 46] Output [1]: [ss_item_sk#38] -(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 +(51) CometBroadcastHashJoin +Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Right output [1]: [ss_item_sk#38] +Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight -(57) BroadcastExchange +(52) CometBroadcastExchange 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] +Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -(58) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#39] -Join type: Inner -Join condition: None +(53) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight -(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] +(54) CometProject 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] +Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -(60) ReusedExchange [Reuses operator id: 127] -Output [1]: [d_date_sk#43] +(55) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#43, d_year#44, d_moy#45] +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 -(61) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#43] -Join type: Inner -Join condition: None +(56) CometFilter +Input [3]: [d_date_sk#43, 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 = 11)) AND isnotnull(d_date_sk#43)) + +(57) CometProject +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Arguments: [d_date_sk#43], [d_date_sk#43] + +(58) CometBroadcastExchange +Input [1]: [d_date_sk#43] +Arguments: [d_date_sk#43] -(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] +(59) CometBroadcastHashJoin +Left 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] +Right output [1]: [d_date_sk#43] +Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight + +(60) CometProject 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] +Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -(63) HashAggregate [codegen id : 11] +(61) CometHashAggregate 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#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#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] +(62) CometColumnarExchange +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] +Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(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] +(63) CometHashAggregate +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] 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))#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 : 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)))) +(64) CometFilter +Input [5]: [i_brand_id#40, i_class_id#41, i_category_id#42, sales#49, number_sales#50] +Condition : (isnotnull(sales#49) AND (cast(sales#49 as decimal(32,6)) > cast(Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) -(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] +(65) CometProject +Input [5]: [i_brand_id#40, i_class_id#41, i_category_id#42, sales#49, number_sales#50] +Arguments: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56], [sales#49, number_sales#50, store AS channel#53, i_brand_id#40 AS i_brand_id#54, i_class_id#41 AS i_class_id#55, i_category_id#42 AS i_category_id#56] -(68) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#60, cs_quantity#61, cs_list_price#62, cs_sold_date_sk#63] +(66) Scan parquet spark_catalog.default.catalog_sales +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#63), dynamicpruningexpression(cs_sold_date_sk#63 IN dynamicpruning#64)] +PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_sold_date_sk#60 IN dynamicpruning#61)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(69) CometFilter -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 : 23] -Input [4]: [cs_item_sk#60, cs_quantity#61, cs_list_price#62, cs_sold_date_sk#63] +(67) CometFilter +Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] +Condition : isnotnull(cs_item_sk#57) -(71) ReusedExchange [Reuses operator id: 50] +(68) ReusedExchange [Reuses operator id: 46] Output [1]: [ss_item_sk#38] -(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#65, i_brand_id#66, i_class_id#67, i_category_id#68] - -(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 : 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: 127] -Output [1]: [d_date_sk#69] - -(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 : 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 : 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#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 : 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 : 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 : 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#81, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#84] +(69) CometBroadcastHashJoin +Left output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] +Right output [1]: [ss_item_sk#38] +Arguments: [cs_item_sk#57], [ss_item_sk#38], LeftSemi, BuildRight + +(70) ReusedExchange [Reuses operator id: 52] +Output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] + +(71) CometBroadcastHashJoin +Left output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] +Right output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] +Arguments: [cs_item_sk#57], [i_item_sk#62], Inner, BuildRight + +(72) CometProject +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] +Arguments: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#63, i_class_id#64, i_category_id#65], [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#63, i_class_id#64, i_category_id#65] + +(73) ReusedExchange [Reuses operator id: 58] +Output [1]: [d_date_sk#66] + +(74) CometBroadcastHashJoin +Left 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] +Right output [1]: [d_date_sk#66] +Arguments: [cs_sold_date_sk#60], [d_date_sk#66], Inner, BuildRight + +(75) CometProject +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] +Arguments: [cs_quantity#58, cs_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65], [cs_quantity#58, cs_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65] + +(76) CometHashAggregate +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)] + +(77) CometColumnarExchange +Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#67, isEmpty#68, count#69] +Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(78) CometHashAggregate +Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#67, isEmpty#68, count#69] +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)] + +(79) CometFilter +Input [5]: [i_brand_id#63, i_class_id#64, i_category_id#65, sales#70, number_sales#71] +Condition : (isnotnull(sales#70) AND (cast(sales#70 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) + +(80) CometProject +Input [5]: [i_brand_id#63, i_class_id#64, i_category_id#65, sales#70, number_sales#71] +Arguments: [sales#70, number_sales#71, channel#72, i_brand_id#63, i_class_id#64, i_category_id#65], [sales#70, number_sales#71, catalog AS channel#72, i_brand_id#63, i_class_id#64, i_category_id#65] + +(81) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#73, ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76] 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#76), dynamicpruningexpression(ws_sold_date_sk#76 IN dynamicpruning#77)] 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) +(82) CometFilter +Input [4]: [ws_item_sk#73, ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76] +Condition : isnotnull(ws_item_sk#73) -(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] +(83) ReusedExchange [Reuses operator id: 46] Output [1]: [ss_item_sk#38] -(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#86, i_brand_id#87, i_class_id#88, i_category_id#89] - -(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 : 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: 127] -Output [1]: [d_date_sk#90] - -(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 : 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 : 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#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 : 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 : 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 : 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 : 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 : 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#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 : 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#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] +(84) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#73, ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76] +Right output [1]: [ss_item_sk#38] +Arguments: [ws_item_sk#73], [ss_item_sk#38], LeftSemi, BuildRight + +(85) ReusedExchange [Reuses operator id: 52] +Output [4]: [i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81] + +(86) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#73, ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76] +Right output [4]: [i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81] +Arguments: [ws_item_sk#73], [i_item_sk#78], Inner, BuildRight + +(87) CometProject +Input [8]: [ws_item_sk#73, ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76, i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81] +Arguments: [ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76, i_brand_id#79, i_class_id#80, i_category_id#81], [ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76, i_brand_id#79, i_class_id#80, i_category_id#81] + +(88) ReusedExchange [Reuses operator id: 58] +Output [1]: [d_date_sk#82] + +(89) CometBroadcastHashJoin +Left output [6]: [ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76, i_brand_id#79, i_class_id#80, i_category_id#81] +Right output [1]: [d_date_sk#82] +Arguments: [ws_sold_date_sk#76], [d_date_sk#82], Inner, BuildRight + +(90) CometProject +Input [7]: [ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76, i_brand_id#79, i_class_id#80, i_category_id#81, d_date_sk#82] +Arguments: [ws_quantity#74, ws_list_price#75, i_brand_id#79, i_class_id#80, i_category_id#81], [ws_quantity#74, ws_list_price#75, i_brand_id#79, i_class_id#80, i_category_id#81] + +(91) CometHashAggregate +Input [5]: [ws_quantity#74, ws_list_price#75, i_brand_id#79, i_class_id#80, i_category_id#81] +Keys [3]: [i_brand_id#79, i_class_id#80, i_category_id#81] +Functions [2]: [partial_sum((cast(ws_quantity#74 as decimal(10,0)) * ws_list_price#75)), partial_count(1)] + +(92) CometColumnarExchange +Input [6]: [i_brand_id#79, i_class_id#80, i_category_id#81, sum#83, isEmpty#84, count#85] +Arguments: hashpartitioning(i_brand_id#79, i_class_id#80, i_category_id#81, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(93) CometHashAggregate +Input [6]: [i_brand_id#79, i_class_id#80, i_category_id#81, sum#83, isEmpty#84, count#85] +Keys [3]: [i_brand_id#79, i_class_id#80, i_category_id#81] +Functions [2]: [sum((cast(ws_quantity#74 as decimal(10,0)) * ws_list_price#75)), count(1)] + +(94) CometFilter +Input [5]: [i_brand_id#79, i_class_id#80, i_category_id#81, sales#86, number_sales#87] +Condition : (isnotnull(sales#86) AND (cast(sales#86 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) + +(95) CometProject +Input [5]: [i_brand_id#79, i_class_id#80, i_category_id#81, sales#86, number_sales#87] +Arguments: [sales#86, number_sales#87, channel#88, i_brand_id#79, i_class_id#80, i_category_id#81], [sales#86, number_sales#87, web AS channel#88, i_brand_id#79, i_class_id#80, i_category_id#81] + +(96) CometUnion +Child 0 Input [6]: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Child 1 Input [6]: [sales#70, number_sales#71, channel#72, i_brand_id#63, i_class_id#64, i_category_id#65] +Child 2 Input [6]: [sales#86, number_sales#87, channel#88, i_brand_id#79, i_class_id#80, i_category_id#81] + +(97) CometExpand +Input [6]: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Arguments: [[sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56, 0], [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, null, 1], [sales#49, number_sales#50, channel#53, i_brand_id#54, null, null, 3], [sales#49, number_sales#50, channel#53, null, null, null, 7], [sales#49, number_sales#50, null, null, null, null, 15]], [sales#49, number_sales#50, channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, spark_grouping_id#93] + +(98) CometHashAggregate +Input [7]: [sales#49, number_sales#50, channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, spark_grouping_id#93] +Keys [5]: [channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, spark_grouping_id#93] +Functions [2]: [partial_sum(sales#49), partial_sum(number_sales#50)] + +(99) CometColumnarExchange +Input [8]: [channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, spark_grouping_id#93, sum#94, isEmpty#95, sum#96] +Arguments: hashpartitioning(channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, spark_grouping_id#93, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(100) CometHashAggregate +Input [8]: [channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, spark_grouping_id#93, sum#94, isEmpty#95, sum#96] +Keys [5]: [channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, spark_grouping_id#93] +Functions [2]: [sum(sales#49), sum(number_sales#50)] + +(101) CometTakeOrderedAndProject +Input [6]: [channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, sum(sales)#97, sum(number_sales)#98] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#89 ASC NULLS FIRST,i_brand_id#90 ASC NULLS FIRST,i_class_id#91 ASC NULLS FIRST,i_category_id#92 ASC NULLS FIRST], output=[channel#89,i_brand_id#90,i_class_id#91,i_category_id#92,sum(sales)#97,sum(number_sales)#98]), [channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, sum(sales)#97, sum(number_sales)#98], 100, [channel#89 ASC NULLS FIRST, i_brand_id#90 ASC NULLS FIRST, i_class_id#91 ASC NULLS FIRST, i_category_id#92 ASC NULLS FIRST], [channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, sum(sales)#97, sum(number_sales)#98] + +(102) ColumnarToRow [codegen id : 1] +Input [6]: [channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, sum(sales)#97, sum(number_sales)#98] ===== Subqueries ===== -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#117, ss_list_price#118, ss_sold_date_sk#119] +Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#51, [id=#52] +* ColumnarToRow (119) ++- CometHashAggregate (118) + +- CometColumnarExchange (117) + +- CometHashAggregate (116) + +- CometUnion (115) + :- CometProject (106) + : +- CometBroadcastHashJoin (105) + : :- CometScan parquet spark_catalog.default.store_sales (103) + : +- ReusedExchange (104) + :- CometProject (110) + : +- CometBroadcastHashJoin (109) + : :- CometScan parquet spark_catalog.default.catalog_sales (107) + : +- ReusedExchange (108) + +- CometProject (114) + +- CometBroadcastHashJoin (113) + :- CometScan parquet spark_catalog.default.web_sales (111) + +- ReusedExchange (112) + + +(103) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_quantity#99, ss_list_price#100, ss_sold_date_sk#101] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#119), dynamicpruningexpression(ss_sold_date_sk#119 IN dynamicpruning#120)] +PartitionFilters: [isnotnull(ss_sold_date_sk#101), dynamicpruningexpression(ss_sold_date_sk#101 IN dynamicpruning#102)] ReadSchema: struct -(107) ReusedExchange [Reuses operator id: 21] -Output [1]: [d_date_sk#121] +(104) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#103] -(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 +(105) CometBroadcastHashJoin +Left output [3]: [ss_quantity#99, ss_list_price#100, ss_sold_date_sk#101] +Right output [1]: [d_date_sk#103] +Arguments: [ss_sold_date_sk#101], [d_date_sk#103], Inner, BuildRight -(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] +(106) CometProject +Input [4]: [ss_quantity#99, ss_list_price#100, ss_sold_date_sk#101, d_date_sk#103] +Arguments: [quantity#104, list_price#105], [ss_quantity#99 AS quantity#104, ss_list_price#100 AS list_price#105] -(110) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#124, cs_list_price#125, cs_sold_date_sk#126] +(107) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#106, cs_list_price#107, cs_sold_date_sk#108] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#126), dynamicpruningexpression(cs_sold_date_sk#126 IN dynamicpruning#127)] +PartitionFilters: [isnotnull(cs_sold_date_sk#108), dynamicpruningexpression(cs_sold_date_sk#108 IN dynamicpruning#109)] ReadSchema: struct -(111) ReusedExchange [Reuses operator id: 21] -Output [1]: [d_date_sk#128] +(108) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#110] -(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 +(109) CometBroadcastHashJoin +Left output [3]: [cs_quantity#106, cs_list_price#107, cs_sold_date_sk#108] +Right output [1]: [d_date_sk#110] +Arguments: [cs_sold_date_sk#108], [d_date_sk#110], Inner, BuildRight -(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] +(110) CometProject +Input [4]: [cs_quantity#106, cs_list_price#107, cs_sold_date_sk#108, d_date_sk#110] +Arguments: [quantity#111, list_price#112], [cs_quantity#106 AS quantity#111, cs_list_price#107 AS list_price#112] -(114) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#131, ws_list_price#132, ws_sold_date_sk#133] +(111) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#133), dynamicpruningexpression(ws_sold_date_sk#133 IN dynamicpruning#134)] +PartitionFilters: [isnotnull(ws_sold_date_sk#115), dynamicpruningexpression(ws_sold_date_sk#115 IN dynamicpruning#116)] ReadSchema: struct -(115) ReusedExchange [Reuses operator id: 21] -Output [1]: [d_date_sk#135] +(112) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#117] -(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 +(113) CometBroadcastHashJoin +Left output [3]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115] +Right output [1]: [d_date_sk#117] +Arguments: [ws_sold_date_sk#115], [d_date_sk#117], Inner, BuildRight -(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] +(114) CometProject +Input [4]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115, d_date_sk#117] +Arguments: [quantity#118, list_price#119], [ws_quantity#113 AS quantity#118, ws_list_price#114 AS list_price#119] -(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] +(115) CometUnion +Child 0 Input [2]: [quantity#104, list_price#105] +Child 1 Input [2]: [quantity#111, list_price#112] +Child 2 Input [2]: [quantity#118, list_price#119] -(119) CometHashAggregate -Input [2]: [quantity#122, list_price#123] +(116) CometHashAggregate +Input [2]: [quantity#104, list_price#105] Keys: [] -Functions [1]: [partial_avg((cast(quantity#122 as decimal(10,0)) * list_price#123))] +Functions [1]: [partial_avg((cast(quantity#104 as decimal(10,0)) * list_price#105))] -(120) ColumnarToRow [codegen id : 1] -Input [2]: [sum#138, count#139] +(117) CometColumnarExchange +Input [2]: [sum#120, count#121] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(121) Exchange -Input [2]: [sum#138, count#139] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] - -(122) HashAggregate [codegen id : 2] -Input [2]: [sum#138, count#139] +(118) CometHashAggregate +Input [2]: [sum#120, count#121] Keys: [] -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] +Functions [1]: [avg((cast(quantity#104 as decimal(10,0)) * list_price#105))] + +(119) ColumnarToRow [codegen id : 1] +Input [1]: [average_sales#122] -Subquery:2 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#119 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 103 Hosting Expression = ss_sold_date_sk#101 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#126 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 107 Hosting Expression = cs_sold_date_sk#108 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 114 Hosting Expression = ws_sold_date_sk#133 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 111 Hosting Expression = ws_sold_date_sk#115 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (127) -+- * ColumnarToRow (126) - +- CometProject (125) - +- CometFilter (124) - +- CometScan parquet spark_catalog.default.date_dim (123) +BroadcastExchange (124) ++- * ColumnarToRow (123) + +- CometProject (122) + +- CometFilter (121) + +- CometScan parquet spark_catalog.default.date_dim (120) -(123) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#43, d_year#142, d_moy#143] +(120) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#43, d_year#44, d_moy#45] 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 -(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)) +(121) CometFilter +Input [3]: [d_date_sk#43, 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 = 11)) AND isnotnull(d_date_sk#43)) -(125) CometProject -Input [3]: [d_date_sk#43, d_year#142, d_moy#143] +(122) CometProject +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] Arguments: [d_date_sk#43], [d_date_sk#43] -(126) ColumnarToRow [codegen id : 1] +(123) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] -(127) BroadcastExchange +(124) BroadcastExchange Input [1]: [d_date_sk#43] -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=7] -Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (132) -+- * ColumnarToRow (131) - +- CometProject (130) - +- CometFilter (129) - +- CometScan parquet spark_catalog.default.date_dim (128) +Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 +BroadcastExchange (129) ++- * ColumnarToRow (128) + +- CometProject (127) + +- CometFilter (126) + +- CometScan parquet spark_catalog.default.date_dim (125) -(128) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#144] +(125) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_year#123] 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 -(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)) +(126) CometFilter +Input [2]: [d_date_sk#26, d_year#123] +Condition : (((isnotnull(d_year#123) AND (d_year#123 >= 1999)) AND (d_year#123 <= 2001)) AND isnotnull(d_date_sk#26)) -(130) CometProject -Input [2]: [d_date_sk#26, d_year#144] +(127) CometProject +Input [2]: [d_date_sk#26, d_year#123] Arguments: [d_date_sk#26], [d_date_sk#26] -(131) ColumnarToRow [codegen id : 1] +(128) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] -(132) BroadcastExchange +(129) BroadcastExchange Input [1]: [d_date_sk#26] -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=8] -Subquery:7 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 +Subquery:7 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 82 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] +Subquery:9 Hosting operator id = 79 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] -Subquery:10 Hosting operator id = 68 Hosting Expression = cs_sold_date_sk#63 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 66 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 98 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] +Subquery:11 Hosting operator id = 94 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] -Subquery:12 Hosting operator id = 84 Hosting Expression = ws_sold_date_sk#84 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 81 Hosting Expression = ws_sold_date_sk#76 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 2257d398f..bb69fc92a 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,190 +1,149 @@ -TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] - 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 (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 (12) - Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] - Filter [sales] - Subquery #3 - WholeStageCodegen (2) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - InputAdapter - Exchange #14 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [sum,count,quantity,list_price] - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,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_quantity,cs_list_price,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_quantity,ws_list_price,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] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales),spark_grouping_id,sum,isEmpty,sum,sum(sales),sum(number_salesL)] + CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum,sales,number_sales] + CometExpand [channel,i_brand_id,i_class_id,i_category_id] [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] + CometUnion [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] + CometProject [i_brand_id,i_class_id,i_category_id] [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] + CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales] + Subquery #3 + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #2 - 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] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,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 #5 - 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 + CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] + CometColumnarExchange #15 + CometHashAggregate [sum,count,quantity,list_price] + CometUnion [quantity,list_price] + CometProject [ss_quantity,ss_list_price] [quantity,list_price] + CometBroadcastHashJoin [ss_quantity,ss_list_price,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_quantity,cs_list_price,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_quantity,ws_list_price,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 + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #2 + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ss_quantity,ss_list_price] + CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk] #4 + CometProject [i_item_sk] [ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,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 [brand_id,class_id,category_id] #5 + CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + CometHashAggregate [brand_id,class_id,category_id] + CometColumnarExchange [brand_id,class_id,category_id] #6 + 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,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 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,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8 - CometBroadcastHashJoin [i_item_sk,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 [i_brand_id,i_class_id,i_category_id] #9 - CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [cs_item_sk,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 - 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 [d_date_sk] #11 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #11 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ws_item_sk,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 - ReusedExchange [d_date_sk] #11 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (9) - BroadcastHashJoin [i_item_sk,ss_item_sk] - 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 - ReusedExchange [ss_item_sk] #4 - InputAdapter - ReusedExchange [d_date_sk] #3 - 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] #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] - Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - BroadcastHashJoin [cs_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - InputAdapter - ReusedExchange [d_date_sk] #3 - 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] #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] - Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - BroadcastHashJoin [ws_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - InputAdapter - ReusedExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8 + CometBroadcastHashJoin [i_item_sk,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 [i_brand_id,i_class_id,i_category_id] #9 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + 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 [d_date_sk] #11 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #11 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #12 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + ReusedExchange [d_date_sk] #11 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] + 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] + ReusedExchange [ss_item_sk] #4 + CometBroadcastExchange [d_date_sk] #14 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] + CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales] + ReusedSubquery [average_sales] #3 + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1)] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #16 + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,cs_quantity,cs_list_price] + CometProject [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,ss_item_sk] + CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [ss_item_sk] #4 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + ReusedExchange [d_date_sk] #14 + CometProject [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] + CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales] + ReusedSubquery [average_sales] #3 + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1)] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #17 + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ws_quantity,ws_list_price] + CometProject [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,ss_item_sk] + CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [ss_item_sk] #4 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + ReusedExchange [d_date_sk] #14 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 38ffcaa3a..199939cd8 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 @@ -1,88 +1,89 @@ == Physical Plan == -TakeOrderedAndProject (84) -+- * BroadcastHashJoin Inner BuildRight (83) - :- * Filter (66) - : +- * HashAggregate (65) - : +- Exchange (64) - : +- * HashAggregate (63) - : +- * Project (62) - : +- * BroadcastHashJoin Inner BuildRight (61) - : :- * Project (59) - : : +- * BroadcastHashJoin Inner BuildRight (58) - : : :- * BroadcastHashJoin LeftSemi BuildRight (51) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (50) - : : : +- * Project (49) - : : : +- * BroadcastHashJoin Inner BuildRight (48) - : : : :- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.item (4) - : : : +- BroadcastExchange (47) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) - : : : :- * HashAggregate (35) - : : : : +- Exchange (34) - : : : : +- * 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) - : : : +- * 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) - : : : +- CometFilter (53) - : : : +- CometScan parquet spark_catalog.default.item (52) - : : +- ReusedExchange (55) - : +- ReusedExchange (60) - +- BroadcastExchange (82) - +- * Filter (81) - +- * HashAggregate (80) - +- Exchange (79) - +- * HashAggregate (78) - +- * Project (77) - +- * BroadcastHashJoin Inner BuildRight (76) - :- * Project (74) - : +- * BroadcastHashJoin Inner BuildRight (73) - : :- * BroadcastHashJoin LeftSemi BuildRight (71) - : : :- * ColumnarToRow (69) - : : : +- CometFilter (68) - : : : +- CometScan parquet spark_catalog.default.store_sales (67) - : : +- ReusedExchange (70) - : +- ReusedExchange (72) - +- ReusedExchange (75) +* ColumnarToRow (85) ++- CometTakeOrderedAndProject (84) + +- CometBroadcastHashJoin (83) + :- CometFilter (64) + : +- CometHashAggregate (63) + : +- CometColumnarExchange (62) + : +- CometHashAggregate (61) + : +- CometProject (60) + : +- CometBroadcastHashJoin (59) + : :- CometProject (54) + : : +- CometBroadcastHashJoin (53) + : : :- CometBroadcastHashJoin (47) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (46) + : : : +- CometProject (45) + : : : +- CometBroadcastHashJoin (44) + : : : :- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.item (3) + : : : +- CometBroadcastExchange (43) + : : : +- CometBroadcastHashJoin (42) + : : : :- CometHashAggregate (32) + : : : : +- CometColumnarExchange (31) + : : : : +- CometHashAggregate (30) + : : : : +- CometProject (29) + : : : : +- CometBroadcastHashJoin (28) + : : : : :- CometProject (26) + : : : : : +- CometBroadcastHashJoin (25) + : : : : : :- CometFilter (6) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (5) + : : : : : +- CometBroadcastExchange (24) + : : : : : +- CometBroadcastHashJoin (23) + : : : : : :- CometFilter (8) + : : : : : : +- CometScan parquet spark_catalog.default.item (7) + : : : : : +- CometBroadcastExchange (22) + : : : : : +- CometProject (21) + : : : : : +- CometBroadcastHashJoin (20) + : : : : : :- CometProject (15) + : : : : : : +- CometBroadcastHashJoin (14) + : : : : : : :- CometFilter (10) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (9) + : : : : : : +- CometBroadcastExchange (13) + : : : : : : +- CometFilter (12) + : : : : : : +- CometScan parquet spark_catalog.default.item (11) + : : : : : +- CometBroadcastExchange (19) + : : : : : +- CometProject (18) + : : : : : +- CometFilter (17) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (16) + : : : : +- ReusedExchange (27) + : : : +- CometBroadcastExchange (41) + : : : +- CometProject (40) + : : : +- CometBroadcastHashJoin (39) + : : : :- CometProject (37) + : : : : +- CometBroadcastHashJoin (36) + : : : : :- CometFilter (34) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (33) + : : : : +- ReusedExchange (35) + : : : +- ReusedExchange (38) + : : +- CometBroadcastExchange (52) + : : +- CometBroadcastHashJoin (51) + : : :- CometFilter (49) + : : : +- CometScan parquet spark_catalog.default.item (48) + : : +- ReusedExchange (50) + : +- CometBroadcastExchange (58) + : +- CometProject (57) + : +- CometFilter (56) + : +- CometScan parquet spark_catalog.default.date_dim (55) + +- CometBroadcastExchange (82) + +- CometFilter (81) + +- CometHashAggregate (80) + +- CometColumnarExchange (79) + +- CometHashAggregate (78) + +- CometProject (77) + +- CometBroadcastHashJoin (76) + :- CometProject (71) + : +- CometBroadcastHashJoin (70) + : :- CometBroadcastHashJoin (68) + : : :- CometFilter (66) + : : : +- CometScan parquet spark_catalog.default.store_sales (65) + : : +- ReusedExchange (67) + : +- ReusedExchange (69) + +- CometBroadcastExchange (75) + +- CometProject (74) + +- CometFilter (73) + +- CometScan parquet spark_catalog.default.date_dim (72) (1) Scan parquet spark_catalog.default.store_sales @@ -97,24 +98,18 @@ 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 : 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 +(3) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) -(6) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(7) Scan parquet spark_catalog.default.store_sales +(5) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] @@ -122,22 +117,22 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(8) CometFilter +(6) CometFilter Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(9) Scan parquet spark_catalog.default.item +(7) 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 -(10) CometFilter +(8) 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)) -(11) Scan parquet spark_catalog.default.catalog_sales +(9) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] @@ -145,116 +140,111 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(12) CometFilter +(10) CometFilter Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) -(13) Scan parquet spark_catalog.default.item +(11) 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 -(14) CometFilter +(12) 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) -(15) CometBroadcastExchange +(13) 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] -(16) CometBroadcastHashJoin +(14) 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 -(17) CometProject +(15) 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 +(16) 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 -(19) CometFilter +(17) 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 +(18) CometProject Input [2]: [d_date_sk#24, d_year#25] Arguments: [d_date_sk#24], [d_date_sk#24] -(21) CometBroadcastExchange +(19) CometBroadcastExchange Input [1]: [d_date_sk#24] Arguments: [d_date_sk#24] -(22) CometBroadcastHashJoin +(20) 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 +(21) 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] -(24) CometBroadcastExchange +(22) CometBroadcastExchange Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] -(25) CometBroadcastHashJoin +(23) 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 -(26) CometBroadcastExchange +(24) CometBroadcastExchange Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(27) CometBroadcastHashJoin +(25) 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 -(28) CometProject +(26) 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] -(29) ReusedExchange [Reuses operator id: 21] +(27) ReusedExchange [Reuses operator id: 19] Output [1]: [d_date_sk#26] -(30) CometBroadcastHashJoin +(28) 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 -(31) CometProject +(29) 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] -(32) CometHashAggregate +(30) CometHashAggregate Input [3]: [brand_id#27, class_id#28, category_id#29] Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] -(33) ColumnarToRow [codegen id : 1] -Input [3]: [brand_id#27, class_id#28, category_id#29] - -(34) Exchange +(31) CometColumnarExchange 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] +Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(35) HashAggregate [codegen id : 3] +(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#27, class_id#28, category_id#29] -(36) Scan parquet spark_catalog.default.web_sales +(33) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] @@ -262,482 +252,492 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(37) CometFilter +(34) CometFilter Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] Condition : isnotnull(ws_item_sk#30) -(38) ReusedExchange [Reuses operator id: 15] +(35) ReusedExchange [Reuses operator id: 13] Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -(39) CometBroadcastHashJoin +(36) 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) CometProject +(37) 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) ReusedExchange [Reuses operator id: 21] +(38) ReusedExchange [Reuses operator id: 19] Output [1]: [d_date_sk#37] -(42) CometBroadcastHashJoin +(39) 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) CometProject +(40) 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) ColumnarToRow [codegen id : 2] -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(45) BroadcastExchange +(41) CometBroadcastExchange 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] +Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] -(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 +(42) CometBroadcastHashJoin +Left output [3]: [brand_id#27, class_id#28, category_id#29] +Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [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)], [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)], LeftSemi, BuildRight -(47) BroadcastExchange +(43) CometBroadcastExchange 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] +Arguments: [brand_id#27, class_id#28, category_id#29] -(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#27, class_id#28, category_id#29] -Join type: Inner -Join condition: None +(44) CometBroadcastHashJoin +Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Right output [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight -(49) Project [codegen id : 4] -Output [1]: [i_item_sk#6 AS ss_item_sk#38] +(45) CometProject 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] +Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] -(50) BroadcastExchange +(46) CometBroadcastExchange Input [1]: [ss_item_sk#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: [ss_item_sk#38] -(51) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#38] -Join type: LeftSemi -Join condition: None +(47) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Right output [1]: [ss_item_sk#38] +Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight -(52) Scan parquet spark_catalog.default.item +(48) Scan parquet spark_catalog.default.item 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 +(49) CometFilter 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 : 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] +(50) ReusedExchange [Reuses operator id: 46] Output [1]: [ss_item_sk#38] -(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 +(51) CometBroadcastHashJoin +Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Right output [1]: [ss_item_sk#38] +Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight -(57) BroadcastExchange +(52) CometBroadcastExchange 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] +Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -(58) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#39] -Join type: Inner -Join condition: None +(53) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight -(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] +(54) CometProject 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] +Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -(60) ReusedExchange [Reuses operator id: 106] -Output [1]: [d_date_sk#43] +(55) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#43, d_week_seq#44] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] +ReadSchema: struct -(61) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#43] -Join type: Inner -Join condition: None +(56) CometFilter +Input [2]: [d_date_sk#43, d_week_seq#44] +Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = ReusedSubquery Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) -(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] +(57) CometProject +Input [2]: [d_date_sk#43, d_week_seq#44] +Arguments: [d_date_sk#43], [d_date_sk#43] + +(58) CometBroadcastExchange +Input [1]: [d_date_sk#43] +Arguments: [d_date_sk#43] + +(59) CometBroadcastHashJoin +Left 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] +Right output [1]: [d_date_sk#43] +Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight + +(60) CometProject 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] +Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -(63) HashAggregate [codegen id : 11] +(61) CometHashAggregate 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#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 +(62) CometColumnarExchange 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] +Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(65) HashAggregate [codegen id : 24] +(63) CometHashAggregate 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))#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 : 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)))) +(64) CometFilter +Input [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, 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)))) -(67) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60] +(65) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#60), dynamicpruningexpression(ss_sold_date_sk#60 IN dynamicpruning#61)] +PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(68) CometFilter -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 : 22] -Input [4]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60] +(66) CometFilter +Input [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] +Condition : isnotnull(ss_item_sk#55) -(70) ReusedExchange [Reuses operator id: 50] +(67) ReusedExchange [Reuses operator id: 46] Output [1]: [ss_item_sk#38] -(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#62, i_brand_id#63, i_class_id#64, i_category_id#65] - -(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 : 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: 120] -Output [1]: [d_date_sk#66] - -(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 : 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 : 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#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 : 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 : 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#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 : 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#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] +(68) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] +Right output [1]: [ss_item_sk#38] +Arguments: [ss_item_sk#55], [ss_item_sk#38], LeftSemi, BuildRight + +(69) ReusedExchange [Reuses operator id: 52] +Output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] + +(70) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] +Right output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] +Arguments: [ss_item_sk#55], [i_item_sk#60], Inner, BuildRight + +(71) CometProject +Input [8]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] +Arguments: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#61, i_class_id#62, i_category_id#63], [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#61, i_class_id#62, i_category_id#63] + +(72) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#64, d_week_seq#65] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] +ReadSchema: struct + +(73) CometFilter +Input [2]: [d_date_sk#64, d_week_seq#65] +Condition : ((isnotnull(d_week_seq#65) AND (d_week_seq#65 = ReusedSubquery Subquery scalar-subquery#66, [id=#67])) AND isnotnull(d_date_sk#64)) + +(74) CometProject +Input [2]: [d_date_sk#64, d_week_seq#65] +Arguments: [d_date_sk#64], [d_date_sk#64] + +(75) CometBroadcastExchange +Input [1]: [d_date_sk#64] +Arguments: [d_date_sk#64] + +(76) CometBroadcastHashJoin +Left output [6]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#61, i_class_id#62, i_category_id#63] +Right output [1]: [d_date_sk#64] +Arguments: [ss_sold_date_sk#58], [d_date_sk#64], Inner, BuildRight + +(77) CometProject +Input [7]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#61, i_class_id#62, i_category_id#63, d_date_sk#64] +Arguments: [ss_quantity#56, ss_list_price#57, i_brand_id#61, i_class_id#62, i_category_id#63], [ss_quantity#56, ss_list_price#57, i_brand_id#61, i_class_id#62, i_category_id#63] + +(78) CometHashAggregate +Input [5]: [ss_quantity#56, ss_list_price#57, i_brand_id#61, i_class_id#62, i_category_id#63] +Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] +Functions [2]: [partial_sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), partial_count(1)] + +(79) CometColumnarExchange +Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] +Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(80) CometHashAggregate +Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] +Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] +Functions [2]: [sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), count(1)] + +(81) CometFilter +Input [6]: [channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] +Condition : (isnotnull(sales#72) AND (cast(sales#72 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) + +(82) CometBroadcastExchange +Input [6]: [channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] +Arguments: [channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] + +(83) CometBroadcastHashJoin +Left output [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] +Right output [6]: [channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] +Arguments: [i_brand_id#40, i_class_id#41, i_category_id#42], [i_brand_id#61, i_class_id#62, i_category_id#63], Inner, BuildRight + +(84) CometTakeOrderedAndProject +Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#50,i_brand_id#40,i_class_id#41,i_category_id#42,sales#51,number_sales#52,channel#71,i_brand_id#61,i_class_id#62,i_category_id#63,sales#72,number_sales#73]), [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73], 100, [i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] + +(85) ColumnarToRow [codegen id : 1] +Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] ===== Subqueries ===== -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#78, ss_list_price#79, ss_sold_date_sk#80] +Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#53, [id=#54] +* ColumnarToRow (102) ++- CometHashAggregate (101) + +- CometColumnarExchange (100) + +- CometHashAggregate (99) + +- CometUnion (98) + :- CometProject (89) + : +- CometBroadcastHashJoin (88) + : :- CometScan parquet spark_catalog.default.store_sales (86) + : +- ReusedExchange (87) + :- CometProject (93) + : +- CometBroadcastHashJoin (92) + : :- CometScan parquet spark_catalog.default.catalog_sales (90) + : +- ReusedExchange (91) + +- CometProject (97) + +- CometBroadcastHashJoin (96) + :- CometScan parquet spark_catalog.default.web_sales (94) + +- ReusedExchange (95) + + +(86) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_quantity#74, ss_list_price#75, ss_sold_date_sk#76] 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#76), dynamicpruningexpression(ss_sold_date_sk#76 IN dynamicpruning#77)] ReadSchema: struct -(86) ReusedExchange [Reuses operator id: 21] -Output [1]: [d_date_sk#82] +(87) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#78] -(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 +(88) CometBroadcastHashJoin +Left output [3]: [ss_quantity#74, ss_list_price#75, ss_sold_date_sk#76] +Right output [1]: [d_date_sk#78] +Arguments: [ss_sold_date_sk#76], [d_date_sk#78], Inner, BuildRight -(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] +(89) CometProject +Input [4]: [ss_quantity#74, ss_list_price#75, ss_sold_date_sk#76, d_date_sk#78] +Arguments: [quantity#79, list_price#80], [ss_quantity#74 AS quantity#79, ss_list_price#75 AS list_price#80] -(89) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87] +(90) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83] 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#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#84)] ReadSchema: struct -(90) ReusedExchange [Reuses operator id: 21] -Output [1]: [d_date_sk#89] +(91) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#85] -(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 +(92) CometBroadcastHashJoin +Left output [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83] +Right output [1]: [d_date_sk#85] +Arguments: [cs_sold_date_sk#83], [d_date_sk#85], Inner, BuildRight -(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] +(93) CometProject +Input [4]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83, d_date_sk#85] +Arguments: [quantity#86, list_price#87], [cs_quantity#81 AS quantity#86, cs_list_price#82 AS list_price#87] -(93) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] +(94) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90] 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#90), dynamicpruningexpression(ws_sold_date_sk#90 IN dynamicpruning#91)] ReadSchema: struct -(94) ReusedExchange [Reuses operator id: 21] -Output [1]: [d_date_sk#96] +(95) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#92] -(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 +(96) CometBroadcastHashJoin +Left output [3]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90] +Right output [1]: [d_date_sk#92] +Arguments: [ws_sold_date_sk#90], [d_date_sk#92], Inner, BuildRight -(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] +(97) CometProject +Input [4]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90, d_date_sk#92] +Arguments: [quantity#93, list_price#94], [ws_quantity#88 AS quantity#93, ws_list_price#89 AS list_price#94] -(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] +(98) CometUnion +Child 0 Input [2]: [quantity#79, list_price#80] +Child 1 Input [2]: [quantity#86, list_price#87] +Child 2 Input [2]: [quantity#93, list_price#94] -(98) CometHashAggregate -Input [2]: [quantity#83, list_price#84] +(99) CometHashAggregate +Input [2]: [quantity#79, list_price#80] Keys: [] -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] +Functions [1]: [partial_avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] -(100) Exchange -Input [2]: [sum#99, count#100] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] +(100) CometColumnarExchange +Input [2]: [sum#95, count#96] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(101) HashAggregate [codegen id : 2] -Input [2]: [sum#99, count#100] +(101) CometHashAggregate +Input [2]: [sum#95, count#96] 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))#101] -Results [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))#101 AS average_sales#102] +Functions [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] -Subquery:2 Hosting operator id = 85 Hosting Expression = ss_sold_date_sk#80 IN dynamicpruning#12 +(102) ColumnarToRow [codegen id : 1] +Input [1]: [average_sales#97] -Subquery:3 Hosting operator id = 89 Hosting Expression = cs_sold_date_sk#87 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 86 Hosting Expression = ss_sold_date_sk#76 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 93 Hosting Expression = ws_sold_date_sk#94 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#12 + +Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#90 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (106) -+- * ColumnarToRow (105) - +- CometProject (104) - +- CometFilter (103) - +- CometScan parquet spark_catalog.default.date_dim (102) +BroadcastExchange (107) ++- * ColumnarToRow (106) + +- CometProject (105) + +- CometFilter (104) + +- CometScan parquet spark_catalog.default.date_dim (103) -(102) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_week_seq#103] +(103) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#43, d_week_seq#44] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(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)) +(104) CometFilter +Input [2]: [d_date_sk#43, d_week_seq#44] +Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) -(104) CometProject -Input [2]: [d_date_sk#43, d_week_seq#103] +(105) CometProject +Input [2]: [d_date_sk#43, d_week_seq#44] Arguments: [d_date_sk#43], [d_date_sk#43] -(105) ColumnarToRow [codegen id : 1] +(106) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] -(106) BroadcastExchange +(107) BroadcastExchange Input [1]: [d_date_sk#43] -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=5] -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) +Subquery:6 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] +* ColumnarToRow (111) ++- CometProject (110) + +- CometFilter (109) + +- CometScan parquet spark_catalog.default.date_dim (108) -(107) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#106, d_year#107, d_moy#108, d_dom#109] +(108) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] 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 -(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)) +(109) CometFilter +Input [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] +Condition : (((((isnotnull(d_year#99) AND isnotnull(d_moy#100)) AND isnotnull(d_dom#101)) AND (d_year#99 = 2000)) AND (d_moy#100 = 12)) AND (d_dom#101 = 11)) -(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] +(110) CometProject +Input [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] +Arguments: [d_week_seq#98], [d_week_seq#98] -(110) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#106] +(111) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#98] -Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (115) -+- * ColumnarToRow (114) - +- CometProject (113) - +- CometFilter (112) - +- CometScan parquet spark_catalog.default.date_dim (111) +Subquery:7 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 +BroadcastExchange (116) ++- * ColumnarToRow (115) + +- CometProject (114) + +- CometFilter (113) + +- CometScan parquet spark_catalog.default.date_dim (112) -(111) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#110] +(112) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_year#102] 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 -(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)) +(113) CometFilter +Input [2]: [d_date_sk#26, d_year#102] +Condition : (((isnotnull(d_year#102) AND (d_year#102 >= 1999)) AND (d_year#102 <= 2001)) AND isnotnull(d_date_sk#26)) -(113) CometProject -Input [2]: [d_date_sk#26, d_year#110] +(114) CometProject +Input [2]: [d_date_sk#26, d_year#102] Arguments: [d_date_sk#26], [d_date_sk#26] -(114) ColumnarToRow [codegen id : 1] +(115) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] -(115) BroadcastExchange +(116) BroadcastExchange Input [1]: [d_date_sk#26] -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=6] + +Subquery:8 Hosting operator id = 9 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 = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 +Subquery:10 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:10 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#55, [id=#56] +Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#53, [id=#54] -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) +Subquery:12 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 +BroadcastExchange (121) ++- * ColumnarToRow (120) + +- CometProject (119) + +- CometFilter (118) + +- CometScan parquet spark_catalog.default.date_dim (117) -(116) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#66, d_week_seq#111] +(117) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#64, d_week_seq#65] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(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)) +(118) CometFilter +Input [2]: [d_date_sk#64, d_week_seq#65] +Condition : ((isnotnull(d_week_seq#65) AND (d_week_seq#65 = Subquery scalar-subquery#66, [id=#67])) AND isnotnull(d_date_sk#64)) -(118) CometProject -Input [2]: [d_date_sk#66, d_week_seq#111] -Arguments: [d_date_sk#66], [d_date_sk#66] +(119) CometProject +Input [2]: [d_date_sk#64, d_week_seq#65] +Arguments: [d_date_sk#64], [d_date_sk#64] -(119) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#66] +(120) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#64] -(120) BroadcastExchange -Input [1]: [d_date_sk#66] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] +(121) BroadcastExchange +Input [1]: [d_date_sk#64] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -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) +Subquery:13 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#66, [id=#67] +* ColumnarToRow (125) ++- CometProject (124) + +- CometFilter (123) + +- CometScan parquet spark_catalog.default.date_dim (122) -(121) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#114, d_year#115, d_moy#116, d_dom#117] +(122) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_week_seq#103, d_year#104, d_moy#105, d_dom#106] 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 -(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)) +(123) CometFilter +Input [4]: [d_week_seq#103, d_year#104, d_moy#105, d_dom#106] +Condition : (((((isnotnull(d_year#104) AND isnotnull(d_moy#105)) AND isnotnull(d_dom#106)) AND (d_year#104 = 1999)) AND (d_moy#105 = 12)) AND (d_dom#106 = 11)) + +(124) CometProject +Input [4]: [d_week_seq#103, d_year#104, d_moy#105, d_dom#106] +Arguments: [d_week_seq#103], [d_week_seq#103] -(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] +(125) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#103] -(124) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#114] +Subquery:14 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] 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 aa49638c7..a51d1d007 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,15 +1,15 @@ -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 (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 (2) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - InputAdapter - Exchange #13 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + CometBroadcastHashJoin [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Subquery #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] + CometColumnarExchange #14 CometHashAggregate [sum,count,quantity,list_price] CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] @@ -27,152 +27,127 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ 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 (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] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (4) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #1 + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ss_quantity,ss_list_price] + CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) 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 #4 - 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 (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,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 - CometBroadcastHashJoin [i_item_sk,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 [i_brand_id,i_class_id,i_category_id] #8 - CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [cs_item_sk,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - 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 [d_date_sk] #10 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #10 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (2) + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometBroadcastExchange [ss_item_sk] #3 + CometProject [i_item_sk] [ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,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 [brand_id,class_id,category_id] #4 + CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + CometHashAggregate [brand_id,class_id,category_id] + CometColumnarExchange [brand_id,class_id,category_id] #5 + 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,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 + CometBroadcastHashJoin [i_item_sk,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 [i_brand_id,i_class_id,i_category_id] #8 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + 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 [d_date_sk] #10 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #10 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #11 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + ReusedExchange [d_date_sk] #10 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] + 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] + ReusedExchange [ss_item_sk] #3 + CometBroadcastExchange [d_date_sk] #13 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #2 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometBroadcastExchange [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] #15 + CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + ReusedSubquery [average_sales] #4 + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #16 + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ss_quantity,ss_list_price] + CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_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 #17 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #6 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ws_item_sk,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - ReusedExchange [d_date_sk] #10 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - BroadcastHashJoin [i_item_sk,ss_item_sk] - 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 - ReusedExchange [ss_item_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - 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] #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] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_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 #16 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - ReusedExchange [ss_item_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - InputAdapter - ReusedExchange [d_date_sk] #16 + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + ReusedExchange [ss_item_sk] #3 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + CometBroadcastExchange [d_date_sk] #18 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #6 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] 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 c4772ea80..af9c0909f 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,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (23) -+- * HashAggregate (22) - +- Exchange (21) - +- * ColumnarToRow (20) +* ColumnarToRow (23) ++- CometTakeOrderedAndProject (22) + +- CometHashAggregate (21) + +- CometColumnarExchange (20) +- CometHashAggregate (19) +- CometProject (18) +- CometBroadcastHashJoin (17) @@ -117,23 +117,21 @@ Input [2]: [cs_sales_price#2, ca_zip#9] Keys [1]: [ca_zip#9] Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#2))] -(20) ColumnarToRow [codegen id : 1] +(20) CometColumnarExchange Input [2]: [ca_zip#9, sum#13] +Arguments: hashpartitioning(ca_zip#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(21) Exchange -Input [2]: [ca_zip#9, sum#13] -Arguments: hashpartitioning(ca_zip#9, 5), ENSURE_REQUIREMENTS, [plan_id=1] - -(22) HashAggregate [codegen id : 2] +(21) CometHashAggregate 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))#14] -Results [2]: [ca_zip#9, MakeDecimal(sum(UnscaledValue(cs_sales_price#2))#14,17,2) AS sum(cs_sales_price)#15] -(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] +(22) CometTakeOrderedAndProject +Input [2]: [ca_zip#9, sum(cs_sales_price)#14] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_zip#9 ASC NULLS FIRST], output=[ca_zip#9,sum(cs_sales_price)#14]), [ca_zip#9, sum(cs_sales_price)#14], 100, [ca_zip#9 ASC NULLS FIRST], [ca_zip#9, sum(cs_sales_price)#14] + +(23) ColumnarToRow [codegen id : 1] +Input [2]: [ca_zip#9, sum(cs_sales_price)#14] ===== Subqueries ===== 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 b876fe4de..989ea0856 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,35 +1,33 @@ -TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] - WholeStageCodegen (2) - HashAggregate [ca_zip,sum] [sum(UnscaledValue(cs_sales_price)),sum(cs_sales_price),sum] - InputAdapter - Exchange [ca_zip] #1 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ca_zip,sum,cs_sales_price] - CometProject [cs_sales_price,ca_zip] - CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,ca_zip,d_date_sk] - CometProject [cs_sales_price,cs_sold_date_sk,ca_zip] - CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_state,ca_zip] - CometProject [cs_sales_price,cs_sold_date_sk,c_current_addr_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk,c_customer_sk,c_current_addr_sk] - CometFilter [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_state,ca_zip] #4 - CometFilter [ca_address_sk,ca_state,ca_zip] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [ca_zip,sum(cs_sales_price)] + CometHashAggregate [ca_zip,sum(cs_sales_price),sum,sum(UnscaledValue(cs_sales_price))] + CometColumnarExchange [ca_zip] #1 + CometHashAggregate [ca_zip,sum,cs_sales_price] + CometProject [cs_sales_price,ca_zip] + CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,ca_zip,d_date_sk] + CometProject [cs_sales_price,cs_sold_date_sk,ca_zip] + CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_state,ca_zip] + CometProject [cs_sales_price,cs_sold_date_sk,c_current_addr_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk,c_customer_sk,c_current_addr_sk] + CometFilter [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange [ca_address_sk,ca_state,ca_zip] #4 + CometFilter [ca_address_sk,ca_state,ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + 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/q16/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt index ccec341ad..ebff95d06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt @@ -1,49 +1,51 @@ == Physical Plan == -* HashAggregate (45) -+- Exchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * SortMergeJoin LeftAnti (19) - : : : :- * Project (13) - : : : : +- * SortMergeJoin LeftSemi (12) - : : : : :- * Sort (6) - : : : : : +- Exchange (5) - : : : : : +- * ColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : +- * Sort (11) - : : : : +- Exchange (10) - : : : : +- * ColumnarToRow (9) - : : : : +- CometProject (8) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (7) - : : : +- * Sort (18) - : : : +- Exchange (17) - : : : +- * ColumnarToRow (16) - : : : +- CometProject (15) - : : : +- CometScan parquet spark_catalog.default.catalog_returns (14) - : : +- BroadcastExchange (24) - : : +- * ColumnarToRow (23) - : : +- CometProject (22) - : : +- CometFilter (21) - : : +- CometScan parquet spark_catalog.default.date_dim (20) - : +- 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.call_center (34) +* HashAggregate (47) ++- * ColumnarToRow (46) + +- CometColumnarExchange (45) + +- RowToColumnar (44) + +- * HashAggregate (43) + +- * HashAggregate (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * SortMergeJoin LeftAnti (19) + : : : :- * Project (13) + : : : : +- * SortMergeJoin LeftSemi (12) + : : : : :- * ColumnarToRow (6) + : : : : : +- CometSort (5) + : : : : : +- CometColumnarExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : +- * ColumnarToRow (11) + : : : : +- CometSort (10) + : : : : +- CometColumnarExchange (9) + : : : : +- CometProject (8) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (7) + : : : +- * ColumnarToRow (18) + : : : +- CometSort (17) + : : : +- CometColumnarExchange (16) + : : : +- CometProject (15) + : : : +- CometScan parquet spark_catalog.default.catalog_returns (14) + : : +- BroadcastExchange (24) + : : +- * ColumnarToRow (23) + : : +- CometProject (22) + : : +- CometFilter (21) + : : +- CometScan parquet spark_catalog.default.date_dim (20) + : +- 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.call_center (34) (1) Scan parquet spark_catalog.default.catalog_sales @@ -61,16 +63,16 @@ Condition : ((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -(4) ColumnarToRow [codegen id : 1] +(4) CometColumnarExchange Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] +Arguments: hashpartitioning(cs_order_number#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(5) Exchange +(5) CometSort Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: hashpartitioning(cs_order_number#5, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_order_number#5 ASC NULLS FIRST] -(6) Sort [codegen id : 2] +(6) ColumnarToRow [codegen id : 1] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: [cs_order_number#5 ASC NULLS FIRST], false, 0 (7) Scan parquet spark_catalog.default.catalog_sales Output [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11] @@ -82,24 +84,24 @@ ReadSchema: struct Input [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11] Arguments: [cs_warehouse_sk#9, cs_order_number#10], [cs_warehouse_sk#9, cs_order_number#10] -(9) ColumnarToRow [codegen id : 3] +(9) CometColumnarExchange Input [2]: [cs_warehouse_sk#9, cs_order_number#10] +Arguments: hashpartitioning(cs_order_number#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(10) Exchange +(10) CometSort Input [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: hashpartitioning(cs_order_number#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: [cs_warehouse_sk#9, cs_order_number#10], [cs_order_number#10 ASC NULLS FIRST] -(11) Sort [codegen id : 4] +(11) ColumnarToRow [codegen id : 2] Input [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: [cs_order_number#10 ASC NULLS FIRST], false, 0 -(12) SortMergeJoin [codegen id : 5] +(12) SortMergeJoin [codegen id : 3] Left keys [1]: [cs_order_number#5] Right keys [1]: [cs_order_number#10] Join type: LeftSemi Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#9) -(13) Project [codegen id : 5] +(13) Project [codegen id : 3] Output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] @@ -113,18 +115,18 @@ ReadSchema: struct Input [2]: [cr_order_number#12, cr_returned_date_sk#13] Arguments: [cr_order_number#12], [cr_order_number#12] -(16) ColumnarToRow [codegen id : 6] +(16) CometColumnarExchange Input [1]: [cr_order_number#12] +Arguments: hashpartitioning(cr_order_number#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(17) Exchange +(17) CometSort Input [1]: [cr_order_number#12] -Arguments: hashpartitioning(cr_order_number#12, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: [cr_order_number#12], [cr_order_number#12 ASC NULLS FIRST] -(18) Sort [codegen id : 7] +(18) ColumnarToRow [codegen id : 4] Input [1]: [cr_order_number#12] -Arguments: [cr_order_number#12 ASC NULLS FIRST], false, 0 -(19) SortMergeJoin [codegen id : 11] +(19) SortMergeJoin [codegen id : 8] Left keys [1]: [cs_order_number#5] Right keys [1]: [cr_order_number#12] Join type: LeftAnti @@ -145,20 +147,20 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2002-02-01)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(23) ColumnarToRow [codegen id : 8] +(23) ColumnarToRow [codegen id : 5] Input [1]: [d_date_sk#14] (24) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(25) BroadcastHashJoin [codegen id : 11] +(25) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_ship_date_sk#1] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(26) Project [codegen id : 11] +(26) Project [codegen id : 8] Output [5]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#14] @@ -177,20 +179,20 @@ Condition : ((isnotnull(ca_state#17) AND (ca_state#17 = GA)) AND isnotnull(ca_ad Input [2]: [ca_address_sk#16, ca_state#17] Arguments: [ca_address_sk#16], [ca_address_sk#16] -(30) ColumnarToRow [codegen id : 9] +(30) ColumnarToRow [codegen id : 6] Input [1]: [ca_address_sk#16] (31) BroadcastExchange Input [1]: [ca_address_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(32) BroadcastHashJoin [codegen id : 11] +(32) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_ship_addr_sk#2] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(33) Project [codegen id : 11] +(33) Project [codegen id : 8] Output [4]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [6]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#16] @@ -209,49 +211,55 @@ Condition : ((isnotnull(cc_county#19) AND (cc_county#19 = Williamson County)) AN Input [2]: [cc_call_center_sk#18, cc_county#19] Arguments: [cc_call_center_sk#18], [cc_call_center_sk#18] -(37) ColumnarToRow [codegen id : 10] +(37) ColumnarToRow [codegen id : 7] Input [1]: [cc_call_center_sk#18] (38) BroadcastExchange Input [1]: [cc_call_center_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(39) BroadcastHashJoin [codegen id : 11] +(39) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_call_center_sk#3] Right keys [1]: [cc_call_center_sk#18] Join type: Inner Join condition: None -(40) Project [codegen id : 11] +(40) Project [codegen id : 8] Output [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [5]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#18] -(41) HashAggregate [codegen id : 11] +(41) HashAggregate [codegen id : 8] Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Keys [1]: [cs_order_number#5] Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#20, sum(UnscaledValue(cs_net_profit#7))#21] Results [3]: [cs_order_number#5, sum#22, sum#23] -(42) HashAggregate [codegen id : 11] +(42) HashAggregate [codegen id : 8] Input [3]: [cs_order_number#5, sum#22, sum#23] Keys [1]: [cs_order_number#5] Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#20, sum(UnscaledValue(cs_net_profit#7))#21] Results [3]: [cs_order_number#5, sum#22, sum#23] -(43) HashAggregate [codegen id : 11] +(43) HashAggregate [codegen id : 8] Input [3]: [cs_order_number#5, sum#22, sum#23] Keys: [] Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#20, sum(UnscaledValue(cs_net_profit#7))#21, count(cs_order_number#5)#24] Results [3]: [sum#22, sum#23, count#25] -(44) Exchange +(44) RowToColumnar Input [3]: [sum#22, sum#23, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(45) HashAggregate [codegen id : 12] +(45) CometColumnarExchange +Input [3]: [sum#22, sum#23, count#25] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(46) ColumnarToRow [codegen id : 9] +Input [3]: [sum#22, sum#23, count#25] + +(47) HashAggregate [codegen id : 9] Input [3]: [sum#22, sum#23, count#25] Keys: [] Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt index 8935abb54..605ce3703 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt @@ -1,74 +1,70 @@ -WholeStageCodegen (12) +WholeStageCodegen (9) HashAggregate [sum,sum,count] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] - InputAdapter - Exchange #1 - WholeStageCodegen (11) - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] - HashAggregate [cs_order_number,cs_ext_ship_cost,cs_net_profit] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] - Project [cs_order_number,cs_ext_ship_cost,cs_net_profit] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - BroadcastHashJoin [cs_ship_addr_sk,ca_address_sk] - Project [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk] - SortMergeJoin [cs_order_number,cr_order_number] - InputAdapter - WholeStageCodegen (5) - Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - SortMergeJoin [cs_order_number,cs_order_number,cs_warehouse_sk,cs_warehouse_sk] - InputAdapter - WholeStageCodegen (2) - Sort [cs_order_number] - InputAdapter - Exchange [cs_order_number] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometColumnarExchange #1 + RowToColumnar + WholeStageCodegen (8) + HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] + HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] + HashAggregate [cs_order_number,cs_ext_ship_cost,cs_net_profit] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] + Project [cs_order_number,cs_ext_ship_cost,cs_net_profit] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + BroadcastHashJoin [cs_ship_addr_sk,ca_address_sk] + Project [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk] + SortMergeJoin [cs_order_number,cr_order_number] + InputAdapter + WholeStageCodegen (3) + Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + SortMergeJoin [cs_order_number,cs_order_number,cs_warehouse_sk,cs_warehouse_sk] + InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometSort [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + CometColumnarExchange [cs_order_number] #2 CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] - InputAdapter - WholeStageCodegen (4) - Sort [cs_order_number] - InputAdapter - Exchange [cs_order_number] #3 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter + InputAdapter + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometSort [cs_warehouse_sk,cs_order_number] + CometColumnarExchange [cs_order_number] #3 CometProject [cs_warehouse_sk,cs_order_number] CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] - InputAdapter - WholeStageCodegen (7) - Sort [cr_order_number] - InputAdapter - Exchange [cr_order_number] #4 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter + InputAdapter + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometSort [cr_order_number] + CometColumnarExchange [cr_order_number] #4 CometProject [cr_order_number] CometScan parquet spark_catalog.default.catalog_returns [cr_order_number,cr_returned_date_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (8) + BroadcastExchange #6 + WholeStageCodegen (6) ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) + BroadcastExchange #7 + WholeStageCodegen (7) ColumnarToRow InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometProject [cc_call_center_sk] - CometFilter [cc_call_center_sk,cc_county] - CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_county] + CometProject [cc_call_center_sk] + CometFilter [cc_call_center_sk,cc_county] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_county] 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 8d9edc0a0..ddf62fe94 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,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * HashAggregate (41) - +- Exchange (40) - +- * ColumnarToRow (39) +* ColumnarToRow (42) ++- CometTakeOrderedAndProject (41) + +- CometHashAggregate (40) + +- CometColumnarExchange (39) +- CometHashAggregate (38) +- CometProject (37) +- CometBroadcastHashJoin (36) @@ -226,23 +226,21 @@ Input [6]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25, i_ 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))] -(39) ColumnarToRow [codegen id : 1] +(39) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(41) HashAggregate [codegen id : 2] +(40) CometHashAggregate 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)#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] -(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] +(41) CometTakeOrderedAndProject +Input [15]: [i_item_id#27, i_item_desc#28, s_state#25, store_sales_quantitycount#47, store_sales_quantityave#48, store_sales_quantitystdev#49, store_sales_quantitycov#50, as_store_returns_quantitycount#51, as_store_returns_quantityave#52, as_store_returns_quantitystdev#53, store_returns_quantitycov#54, catalog_sales_quantitycount#55, catalog_sales_quantityave#56, catalog_sales_quantitystdev#57, catalog_sales_quantitycov#58] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#27 ASC NULLS FIRST,i_item_desc#28 ASC NULLS FIRST,s_state#25 ASC NULLS FIRST], output=[i_item_id#27,i_item_desc#28,s_state#25,store_sales_quantitycount#47,store_sales_quantityave#48,store_sales_quantitystdev#49,store_sales_quantitycov#50,as_store_returns_quantitycount#51,as_store_returns_quantityave#52,as_store_returns_quantitystdev#53,store_returns_quantitycov#54,catalog_sales_quantitycount#55,catalog_sales_quantityave#56,catalog_sales_quantitystdev#57,catalog_sales_quantitycov#58]), [i_item_id#27, i_item_desc#28, s_state#25, store_sales_quantitycount#47, store_sales_quantityave#48, store_sales_quantitystdev#49, store_sales_quantitycov#50, as_store_returns_quantitycount#51, as_store_returns_quantityave#52, as_store_returns_quantitystdev#53, store_returns_quantitycov#54, catalog_sales_quantitycount#55, catalog_sales_quantityave#56, catalog_sales_quantitystdev#57, catalog_sales_quantitycov#58], 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#47, store_sales_quantityave#48, store_sales_quantitystdev#49, store_sales_quantitycov#50, as_store_returns_quantitycount#51, as_store_returns_quantityave#52, as_store_returns_quantitystdev#53, store_returns_quantitycov#54, catalog_sales_quantitycount#55, catalog_sales_quantityave#56, catalog_sales_quantitystdev#57, catalog_sales_quantitycov#58] + +(42) ColumnarToRow [codegen id : 1] +Input [15]: [i_item_id#27, i_item_desc#28, s_state#25, store_sales_quantitycount#47, store_sales_quantityave#48, store_sales_quantitystdev#49, store_sales_quantitycov#50, as_store_returns_quantitycount#51, as_store_returns_quantityave#52, as_store_returns_quantitystdev#53, store_returns_quantitycov#54, catalog_sales_quantitycount#55, catalog_sales_quantityave#56, catalog_sales_quantitystdev#57, catalog_sales_quantitycov#58] ===== Subqueries ===== 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 ae6bab279..e350dfe51 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,63 +1,61 @@ -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 (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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [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,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,ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_sk,i_item_id,i_item_desc] - CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_state] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,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 [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_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_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_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 [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,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 - CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - 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_date_sk,d_quarter_name] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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 [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange [d_date_sk] #7 +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [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] + CometHashAggregate [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,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))] + CometColumnarExchange [i_item_id,i_item_desc,s_state] #1 + CometHashAggregate [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,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,ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_sk,i_item_id,i_item_desc] + CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_state] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,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 [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_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_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_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 [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,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_quarter_name] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + 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_date_sk,d_quarter_name] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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 [d_date_sk] #6 CometProject [d_date_sk] CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [s_store_sk,s_state] #8 - CometFilter [s_store_sk,s_state] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] + CometBroadcastExchange [d_date_sk] #7 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_quarter_name] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange [s_store_sk,s_state] #8 + CometFilter [s_store_sk,s_state] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 + CometFilter [i_item_sk,i_item_id,i_item_desc] + 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 c09ef1445..6c969939d 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,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (41) -+- * HashAggregate (40) - +- Exchange (39) - +- * ColumnarToRow (38) +* ColumnarToRow (41) ++- CometTakeOrderedAndProject (40) + +- CometHashAggregate (39) + +- CometColumnarExchange (38) +- CometHashAggregate (37) +- CometExpand (36) +- CometProject (35) @@ -219,23 +219,21 @@ Input [12]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, 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)))] -(38) ColumnarToRow [codegen id : 1] +(38) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(39) CometHashAggregate 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)))#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] -(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] +(40) CometTakeOrderedAndProject +Input [11]: [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, agg1#48, agg2#49, agg3#50, agg4#51, agg5#52, agg6#53, agg7#54] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[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], output=[i_item_id#29,ca_country#30,ca_state#31,ca_county#32,agg1#48,agg2#49,agg3#50,agg4#51,agg5#52,agg6#53,agg7#54]), [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, agg1#48, agg2#49, agg3#50, agg4#51, agg5#52, agg6#53, agg7#54], 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#48, agg2#49, agg3#50, agg4#51, agg5#52, agg6#53, agg7#54] + +(41) ColumnarToRow [codegen id : 1] +Input [11]: [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, agg1#48, agg2#49, agg3#50, agg4#51, agg5#52, agg6#53, agg7#54] ===== Subqueries ===== 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 2519d58a2..8182da775 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,53 +1,51 @@ -TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [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,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,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,i_item_sk,i_item_id] - 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_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,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 [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,ca_address_sk,ca_county,ca_state,ca_country] - 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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - 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_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,cd_demo_sk,cd_dep_count] - CometFilter [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] - 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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange [cd_demo_sk] #5 - CometFilter [cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #8 - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,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)))] + CometColumnarExchange [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] #1 + CometHashAggregate [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,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,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,i_item_sk,i_item_id] + 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_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,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 [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,ca_address_sk,ca_county,ca_state,ca_country] + 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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + 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_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,cd_demo_sk,cd_dep_count] + CometFilter [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] + 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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 + CometProject [cd_demo_sk,cd_dep_count] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometBroadcastExchange [cd_demo_sk] #5 + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] + CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] + CometBroadcastExchange [d_date_sk] #7 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_item_id] #8 + CometFilter [i_item_sk,i_item_id] + 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 a00474bee..2b3480d80 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,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (34) -+- * HashAggregate (33) - +- Exchange (32) - +- * ColumnarToRow (31) +* ColumnarToRow (34) ++- CometTakeOrderedAndProject (33) + +- CometHashAggregate (32) + +- CometColumnarExchange (31) +- CometHashAggregate (30) +- CometProject (29) +- CometBroadcastHashJoin (28) @@ -180,21 +180,19 @@ Input [5]: [ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i 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))] -(31) ColumnarToRow [codegen id : 1] +(31) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(33) HashAggregate [codegen id : 2] +(32) CometHashAggregate 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))#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] -(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] +(33) CometTakeOrderedAndProject +Input [5]: [brand_id#22, brand#23, i_manufact_id#12, i_manufact#13, ext_price#24] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#24 DESC NULLS LAST,brand#23 ASC NULLS FIRST,brand_id#22 ASC NULLS FIRST,i_manufact_id#12 ASC NULLS FIRST,i_manufact#13 ASC NULLS FIRST], output=[brand_id#22,brand#23,i_manufact_id#12,i_manufact#13,ext_price#24]), [brand_id#22, brand#23, i_manufact_id#12, i_manufact#13, ext_price#24], 100, [ext_price#24 DESC NULLS LAST, brand#23 ASC NULLS FIRST, brand_id#22 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST, i_manufact#13 ASC NULLS FIRST], [brand_id#22, brand#23, i_manufact_id#12, i_manufact#13, ext_price#24] + +(34) ColumnarToRow [codegen id : 1] +Input [5]: [brand_id#22, brand#23, i_manufact_id#12, i_manufact#13, ext_price#24] 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 163a31f47..8f81e32ff 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,38 +1,36 @@ -TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,sum,ss_ext_sales_price] - CometProject [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip,s_store_sk,s_zip] - CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk,ca_address_sk,ca_zip] - 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,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_customer_sk,c_current_addr_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,ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_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 [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] #3 - CometProject [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #4 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_zip] #5 - CometFilter [ca_address_sk,ca_zip] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange [s_store_sk,s_zip] #6 - CometFilter [s_store_sk,s_zip] - CometScan parquet spark_catalog.default.store [s_store_sk,s_zip] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [brand_id,brand,i_manufact_id,i_manufact,ext_price] + CometHashAggregate [brand_id,brand,i_manufact_id,i_manufact,ext_price,i_brand,i_brand_id,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange [i_brand,i_brand_id,i_manufact_id,i_manufact] #1 + CometHashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,sum,ss_ext_sales_price] + CometProject [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip,s_store_sk,s_zip] + CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk,ca_address_sk,ca_zip] + 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,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_customer_sk,c_current_addr_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,ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] + CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_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 [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] #3 + CometProject [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #4 + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange [ca_address_sk,ca_zip] #5 + CometFilter [ca_address_sk,ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] + CometBroadcastExchange [s_store_sk,s_zip] #6 + CometFilter [s_store_sk,s_zip] + 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 5c6b73c4c..10ce239de 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,39 +1,37 @@ == Physical Plan == -* 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) +* ColumnarToRow (33) ++- CometSort (32) + +- CometColumnarExchange (31) + +- CometProject (30) + +- CometBroadcastHashJoin (29) + :- CometProject (19) + : +- CometBroadcastHashJoin (18) + : :- CometHashAggregate (13) + : : +- CometColumnarExchange (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) + : +- CometBroadcastExchange (17) + : +- CometProject (16) + : +- CometFilter (15) + : +- CometScan parquet spark_catalog.default.date_dim (14) + +- CometBroadcastExchange (28) + +- CometProject (27) + +- CometBroadcastHashJoin (26) + :- CometHashAggregate (21) + : +- ReusedExchange (20) + +- CometBroadcastExchange (25) + +- CometProject (24) + +- CometFilter (23) + +- CometScan parquet spark_catalog.default.date_dim (22) (1) Scan parquet spark_catalog.default.web_sales @@ -91,113 +89,100 @@ 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))] -(12) ColumnarToRow [codegen id : 1] +(12) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(14) HashAggregate [codegen id : 6] +(13) CometHashAggregate 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))#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] -(15) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#33, d_year#34] +(14) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_week_seq#19, d_year#20] 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 -(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)) - -(17) CometProject -Input [2]: [d_week_seq#33, d_year#34] -Arguments: [d_week_seq#33], [d_week_seq#33] +(15) CometFilter +Input [2]: [d_week_seq#19, d_year#20] +Condition : ((isnotnull(d_year#20) AND (d_year#20 = 2001)) AND isnotnull(d_week_seq#19)) -(18) ColumnarToRow [codegen id : 2] -Input [1]: [d_week_seq#33] +(16) CometProject +Input [2]: [d_week_seq#19, d_year#20] +Arguments: [d_week_seq#19], [d_week_seq#19] -(19) BroadcastExchange -Input [1]: [d_week_seq#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(17) CometBroadcastExchange +Input [1]: [d_week_seq#19] +Arguments: [d_week_seq#19] -(20) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [d_week_seq#10] -Right keys [1]: [d_week_seq#33] -Join type: Inner -Join condition: None +(18) CometBroadcastHashJoin +Left output [8]: [d_week_seq#10, sun_sales#21, mon_sales#22, tue_sales#23, wed_sales#24, thu_sales#25, fri_sales#26, sat_sales#27] +Right output [1]: [d_week_seq#19] +Arguments: [d_week_seq#10], [d_week_seq#19], Inner, BuildRight -(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] +(19) CometProject +Input [9]: [d_week_seq#10, sun_sales#21, mon_sales#22, tue_sales#23, wed_sales#24, thu_sales#25, fri_sales#26, sat_sales#27, d_week_seq#19] +Arguments: [d_week_seq1#28, sun_sales1#29, mon_sales1#30, tue_sales1#31, wed_sales1#32, thu_sales1#33, fri_sales1#34, sat_sales1#35], [d_week_seq#10 AS d_week_seq1#28, sun_sales#21 AS sun_sales1#29, mon_sales#22 AS mon_sales1#30, tue_sales#23 AS tue_sales1#31, wed_sales#24 AS wed_sales1#32, thu_sales#25 AS thu_sales1#33, fri_sales#26 AS fri_sales1#34, sat_sales#27 AS sat_sales1#35] -(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] +(20) ReusedExchange [Reuses operator id: 12] +Output [8]: [d_week_seq#10, sum#36, sum#37, sum#38, sum#39, sum#40, sum#41, sum#42] -(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] +(21) CometHashAggregate +Input [8]: [d_week_seq#10, sum#36, sum#37, sum#38, sum#39, sum#40, sum#41, sum#42] 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))#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] -(24) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#50, d_year#51] +(22) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_week_seq#43, d_year#44] 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 -(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)) +(23) CometFilter +Input [2]: [d_week_seq#43, d_year#44] +Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2002)) AND isnotnull(d_week_seq#43)) -(26) CometProject -Input [2]: [d_week_seq#50, d_year#51] -Arguments: [d_week_seq#50], [d_week_seq#50] +(24) CometProject +Input [2]: [d_week_seq#43, d_year#44] +Arguments: [d_week_seq#43], [d_week_seq#43] -(27) ColumnarToRow [codegen id : 4] -Input [1]: [d_week_seq#50] +(25) CometBroadcastExchange +Input [1]: [d_week_seq#43] +Arguments: [d_week_seq#43] -(28) BroadcastExchange -Input [1]: [d_week_seq#50] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(26) CometBroadcastHashJoin +Left output [8]: [d_week_seq#10, sun_sales#21, mon_sales#22, tue_sales#23, wed_sales#24, thu_sales#25, fri_sales#26, sat_sales#27] +Right output [1]: [d_week_seq#43] +Arguments: [d_week_seq#10], [d_week_seq#43], Inner, BuildRight -(29) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [d_week_seq#10] -Right keys [1]: [d_week_seq#50] -Join type: Inner -Join condition: None +(27) CometProject +Input [9]: [d_week_seq#10, sun_sales#21, mon_sales#22, tue_sales#23, wed_sales#24, thu_sales#25, fri_sales#26, sat_sales#27, d_week_seq#43] +Arguments: [d_week_seq2#45, sun_sales2#46, mon_sales2#47, tue_sales2#48, wed_sales2#49, thu_sales2#50, fri_sales2#51, sat_sales2#52], [d_week_seq#10 AS d_week_seq2#45, sun_sales#21 AS sun_sales2#46, mon_sales#22 AS mon_sales2#47, tue_sales#23 AS tue_sales2#48, wed_sales#24 AS wed_sales2#49, thu_sales#25 AS thu_sales2#50, fri_sales#26 AS fri_sales2#51, sat_sales#27 AS sat_sales2#52] -(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] +(28) CometBroadcastExchange +Input [8]: [d_week_seq2#45, sun_sales2#46, mon_sales2#47, tue_sales2#48, wed_sales2#49, thu_sales2#50, fri_sales2#51, sat_sales2#52] +Arguments: [d_week_seq2#45, sun_sales2#46, mon_sales2#47, tue_sales2#48, wed_sales2#49, thu_sales2#50, fri_sales2#51, sat_sales2#52] -(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] +(29) CometBroadcastHashJoin +Left output [8]: [d_week_seq1#28, sun_sales1#29, mon_sales1#30, tue_sales1#31, wed_sales1#32, thu_sales1#33, fri_sales1#34, sat_sales1#35] +Right output [8]: [d_week_seq2#45, sun_sales2#46, mon_sales2#47, tue_sales2#48, wed_sales2#49, thu_sales2#50, fri_sales2#51, sat_sales2#52] +Arguments: [d_week_seq1#28], [(d_week_seq2#45 - 53)], Inner, BuildRight -(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 +(30) CometProject +Input [16]: [d_week_seq1#28, sun_sales1#29, mon_sales1#30, tue_sales1#31, wed_sales1#32, thu_sales1#33, fri_sales1#34, sat_sales1#35, d_week_seq2#45, sun_sales2#46, mon_sales2#47, tue_sales2#48, wed_sales2#49, thu_sales2#50, fri_sales2#51, sat_sales2#52] +Arguments: [d_week_seq1#28, round((sun_sales1 / sun_sales2), 2)#53, round((mon_sales1 / mon_sales2), 2)#54, round((tue_sales1 / tue_sales2), 2)#55, round((wed_sales1 / wed_sales2), 2)#56, round((thu_sales1 / thu_sales2), 2)#57, round((fri_sales1 / fri_sales2), 2)#58, round((sat_sales1 / sat_sales2), 2)#59], [d_week_seq1#28, round((sun_sales1#29 / sun_sales2#46), 2) AS round((sun_sales1 / sun_sales2), 2)#53, round((mon_sales1#30 / mon_sales2#47), 2) AS round((mon_sales1 / mon_sales2), 2)#54, round((tue_sales1#31 / tue_sales2#48), 2) AS round((tue_sales1 / tue_sales2), 2)#55, round((wed_sales1#32 / wed_sales2#49), 2) AS round((wed_sales1 / wed_sales2), 2)#56, round((thu_sales1#33 / thu_sales2#50), 2) AS round((thu_sales1 / thu_sales2), 2)#57, round((fri_sales1#34 / fri_sales2#51), 2) AS round((fri_sales1 / fri_sales2), 2)#58, round((sat_sales1#35 / sat_sales2#52), 2) AS round((sat_sales1 / sat_sales2), 2)#59] -(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] +(31) CometColumnarExchange +Input [8]: [d_week_seq1#28, round((sun_sales1 / sun_sales2), 2)#53, round((mon_sales1 / mon_sales2), 2)#54, round((tue_sales1 / tue_sales2), 2)#55, round((wed_sales1 / wed_sales2), 2)#56, round((thu_sales1 / thu_sales2), 2)#57, round((fri_sales1 / fri_sales2), 2)#58, round((sat_sales1 / sat_sales2), 2)#59] +Arguments: rangepartitioning(d_week_seq1#28 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(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] +(32) CometSort +Input [8]: [d_week_seq1#28, round((sun_sales1 / sun_sales2), 2)#53, round((mon_sales1 / mon_sales2), 2)#54, round((tue_sales1 / tue_sales2), 2)#55, round((wed_sales1 / wed_sales2), 2)#56, round((thu_sales1 / thu_sales2), 2)#57, round((fri_sales1 / fri_sales2), 2)#58, round((sat_sales1 / sat_sales2), 2)#59] +Arguments: [d_week_seq1#28, round((sun_sales1 / sun_sales2), 2)#53, round((mon_sales1 / mon_sales2), 2)#54, round((tue_sales1 / tue_sales2), 2)#55, round((wed_sales1 / wed_sales2), 2)#56, round((thu_sales1 / thu_sales2), 2)#57, round((fri_sales1 / fri_sales2), 2)#58, round((sat_sales1 / sat_sales2), 2)#59], [d_week_seq1#28 ASC NULLS FIRST] -(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 +(33) ColumnarToRow [codegen id : 1] +Input [8]: [d_week_seq1#28, round((sun_sales1 / sun_sales2), 2)#53, round((mon_sales1 / mon_sales2), 2)#54, round((tue_sales1 / tue_sales2), 2)#55, round((wed_sales1 / wed_sales2), 2)#56, round((thu_sales1 / thu_sales2), 2)#57, round((fri_sales1 / fri_sales2), 2)#58, round((sat_sales1 / sat_sales2), 2)#59] 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 5d0658192..c5c06a0b6 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,50 +1,35 @@ -WholeStageCodegen (7) - Sort [d_week_seq1] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [d_week_seq1] #1 - 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] - 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] - InputAdapter - Exchange [d_week_seq] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum,d_day_name,sales_price] - CometProject [sales_price,d_week_seq,d_day_name] - CometBroadcastHashJoin [sold_date_sk,sales_price,d_date_sk,d_week_seq,d_day_name] - CometUnion [sold_date_sk,sales_price] - 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] - CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 - CometFilter [d_date_sk,d_week_seq,d_day_name] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year] - CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] - InputAdapter - BroadcastExchange #5 - 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] - InputAdapter - ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year] - CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] + CometSort [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] + CometColumnarExchange [d_week_seq1] #1 + CometProject [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] [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] + CometBroadcastHashJoin [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] + CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] + CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] + CometHashAggregate [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,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))] + CometColumnarExchange [d_week_seq] #2 + CometHashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum,d_day_name,sales_price] + CometProject [sales_price,d_week_seq,d_day_name] + CometBroadcastHashJoin [sold_date_sk,sales_price,d_date_sk,d_week_seq,d_day_name] + CometUnion [sold_date_sk,sales_price] + 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] + CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 + CometFilter [d_date_sk,d_week_seq,d_day_name] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] + CometBroadcastExchange [d_week_seq] #4 + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] + CometBroadcastExchange [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 + CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] + CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] + CometHashAggregate [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,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))] + ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 + CometBroadcastExchange [d_week_seq] #6 + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] 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 7c29ff218..c6982f073 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 @@ -2,11 +2,11 @@ TakeOrderedAndProject (22) +- * Project (21) +- Window (20) - +- * Sort (19) - +- Exchange (18) - +- * HashAggregate (17) - +- Exchange (16) - +- * ColumnarToRow (15) + +- * ColumnarToRow (19) + +- CometSort (18) + +- CometColumnarExchange (17) + +- CometHashAggregate (16) + +- CometColumnarExchange (15) +- CometHashAggregate (14) +- CometProject (13) +- CometBroadcastHashJoin (12) @@ -92,39 +92,37 @@ Input [6]: [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, 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))] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(16) CometHashAggregate 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] -(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=2] +(17) CometColumnarExchange +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(18) CometSort +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] +Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6], [i_class#9 ASC NULLS FIRST] -(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 +(19) ColumnarToRow [codegen id : 1] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] (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] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] +Arguments: [sum(_w0#15) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#16], [i_class#9] -(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] +(21) Project [codegen id : 2] +Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, ((_w0#15 * 100) / _we0#16) AS revenueratio#17, i_item_id#6] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6, _we0#16] (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] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17, 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#17 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] ===== Subqueries ===== 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 08088a386..9c30689f6 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,38 +1,34 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (4) + WholeStageCodegen (2) Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] InputAdapter Window [_w0,i_class] - WholeStageCodegen (3) - Sort [i_class] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [i_class] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,cs_ext_sales_price] - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometColumnarExchange [i_class] #1 + CometHashAggregate [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum,sum(UnscaledValue(cs_ext_sales_price))] + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,cs_ext_sales_price] + CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + 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 21c979264..ff0f63b74 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 @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * Filter (23) - +- * HashAggregate (22) - +- Exchange (21) - +- * ColumnarToRow (20) +* ColumnarToRow (24) ++- CometTakeOrderedAndProject (23) + +- CometFilter (22) + +- CometHashAggregate (21) + +- CometColumnarExchange (20) +- CometHashAggregate (19) +- CometProject (18) +- CometBroadcastHashJoin (17) @@ -118,27 +118,25 @@ 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)] -(20) ColumnarToRow [codegen id : 1] +(20) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(21) Exchange -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 : 2] +(21) CometHashAggregate 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)#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 : 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) +(22) CometFilter +Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#15, inv_after#16] +Condition : (CASE WHEN (inv_before#15 > 0) THEN ((cast(inv_after#16 as double) / cast(inv_before#15 as double)) >= 0.666667) END AND CASE WHEN (inv_before#15 > 0) THEN ((cast(inv_after#16 as double) / cast(inv_before#15 as double)) <= 1.5) END) + +(23) CometTakeOrderedAndProject +Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#15, inv_after#16] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_warehouse_name#7 ASC NULLS FIRST,i_item_id#9 ASC NULLS FIRST], output=[w_warehouse_name#7,i_item_id#9,inv_before#15,inv_after#16]), [w_warehouse_name#7, i_item_id#9, inv_before#15, inv_after#16], 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#15, inv_after#16] -(24) TakeOrderedAndProject -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] +(24) ColumnarToRow [codegen id : 1] +Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#15, inv_after#16] ===== Subqueries ===== 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 bd81a3a7b..1a4f175c4 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,35 +1,33 @@ -TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [w_warehouse_name,i_item_id,sum,sum,d_date,inv_quantity_on_hand] - CometProject [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] - CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id,d_date_sk,d_date] - CometProject [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id] - CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_sk,i_item_id] - CometProject [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_sk,w_warehouse_name] - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #3 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [i_item_sk,i_item_id] #4 - CometProject [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id,i_current_price] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - CometBroadcastExchange [d_date_sk,d_date] #5 - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] + CometFilter [w_warehouse_name,i_item_id,inv_before,inv_after] + CometHashAggregate [w_warehouse_name,i_item_id,inv_before,inv_after,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)] + CometColumnarExchange [w_warehouse_name,i_item_id] #1 + CometHashAggregate [w_warehouse_name,i_item_id,sum,sum,d_date,inv_quantity_on_hand] + CometProject [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id,d_date_sk,d_date] + CometProject [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_sk,i_item_id] + CometProject [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #3 + CometFilter [w_warehouse_sk,w_warehouse_name] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange [i_item_sk,i_item_id] #4 + CometProject [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id,i_current_price] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] + CometBroadcastExchange [d_date_sk,d_date] #5 + CometFilter [d_date_sk,d_date] + 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 df6a80179..1f7209c62 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,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * HashAggregate (23) - +- Exchange (22) - +- * ColumnarToRow (21) +* ColumnarToRow (24) ++- CometTakeOrderedAndProject (23) + +- CometHashAggregate (22) + +- CometColumnarExchange (21) +- CometHashAggregate (20) +- CometExpand (19) +- CometProject (18) @@ -122,23 +122,21 @@ Input [6]: [inv_quantity_on_hand#3, i_product_name#14, i_brand#15, i_class#16, i 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)] -(21) ColumnarToRow [codegen id : 1] +(21) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(23) HashAggregate [codegen id : 2] +(22) CometHashAggregate 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)#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] -(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] +(23) CometTakeOrderedAndProject +Input [5]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, qoh#21] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#21 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], output=[i_product_name#14,i_brand#15,i_class#16,i_category#17,qoh#21]), [i_product_name#14, i_brand#15, i_class#16, i_category#17, qoh#21], 100, [qoh#21 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#21] + +(24) ColumnarToRow [codegen id : 1] +Input [5]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, qoh#21] ===== Subqueries ===== 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 7d36dc400..05d02283b 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,36 +1,34 @@ -TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count,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,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] - CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] - CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange [w_warehouse_sk] #5 - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_product_name,i_brand,i_class,i_category,qoh] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh,spark_grouping_id,sum,count,avg(inv_quantity_on_hand)] + CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 + CometHashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count,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,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] + CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] + CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometBroadcastExchange [w_warehouse_sk] #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 560a05db9..107640c7a 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,71 +1,71 @@ == Physical Plan == -* 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 (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) +* ColumnarToRow (67) ++- CometHashAggregate (66) + +- CometColumnarExchange (65) + +- CometHashAggregate (64) + +- CometUnion (63) + :- CometProject (46) + : +- CometBroadcastHashJoin (45) + : :- CometProject (40) + : : +- CometSortMergeJoin (39) + : : :- CometSort (24) + : : : +- CometColumnarExchange (23) + : : : +- CometProject (22) + : : : +- CometBroadcastHashJoin (21) + : : : :- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : +- CometBroadcastExchange (20) + : : : +- CometProject (19) + : : : +- CometFilter (18) + : : : +- CometHashAggregate (17) + : : : +- CometColumnarExchange (16) + : : : +- CometHashAggregate (15) + : : : +- CometProject (14) + : : : +- CometBroadcastHashJoin (13) + : : : :- CometProject (9) + : : : : +- CometBroadcastHashJoin (8) + : : : : :- CometFilter (3) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (2) + : : : : +- CometBroadcastExchange (7) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.date_dim (4) + : : : +- CometBroadcastExchange (12) + : : : +- CometFilter (11) + : : : +- CometScan parquet spark_catalog.default.item (10) + : : +- CometSort (38) + : : +- CometProject (37) + : : +- CometFilter (36) + : : +- CometHashAggregate (35) + : : +- CometColumnarExchange (34) + : : +- CometHashAggregate (33) + : : +- CometProject (32) + : : +- CometBroadcastHashJoin (31) + : : :- CometProject (27) + : : : +- CometFilter (26) + : : : +- CometScan parquet spark_catalog.default.store_sales (25) + : : +- CometBroadcastExchange (30) + : : +- CometFilter (29) + : : +- CometScan parquet spark_catalog.default.customer (28) + : +- CometBroadcastExchange (44) + : +- CometProject (43) + : +- CometFilter (42) + : +- CometScan parquet spark_catalog.default.date_dim (41) + +- CometProject (62) + +- CometBroadcastHashJoin (61) + :- CometProject (59) + : +- CometSortMergeJoin (58) + : :- CometSort (52) + : : +- CometColumnarExchange (51) + : : +- CometProject (50) + : : +- CometBroadcastHashJoin (49) + : : :- CometScan parquet spark_catalog.default.web_sales (47) + : : +- ReusedExchange (48) + : +- CometSort (57) + : +- CometProject (56) + : +- CometFilter (55) + : +- CometHashAggregate (54) + : +- ReusedExchange (53) + +- ReusedExchange (60) (1) Scan parquet spark_catalog.default.catalog_sales @@ -75,10 +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 : 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 +(2) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_item_sk#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] @@ -86,203 +83,206 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(4) CometFilter +(3) CometFilter Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] Condition : isnotnull(ss_item_sk#7) -(5) Scan parquet spark_catalog.default.date_dim +(4) 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) CometFilter +(5) 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) CometProject +(6) 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 +(7) CometBroadcastExchange Input [2]: [d_date_sk#10, d_date#11] Arguments: [d_date_sk#10, d_date#11] -(9) CometBroadcastHashJoin +(8) 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 +(9) 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] -(11) Scan parquet spark_catalog.default.item +(10) 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 -(12) CometFilter +(11) CometFilter Input [2]: [i_item_sk#13, i_item_desc#14] Condition : isnotnull(i_item_sk#13) -(13) CometBroadcastExchange +(12) CometBroadcastExchange Input [2]: [i_item_sk#13, i_item_desc#14] Arguments: [i_item_sk#13, i_item_desc#14] -(14) CometBroadcastHashJoin +(13) 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) CometProject +(14) 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) CometHashAggregate +(15) 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)] -(17) ColumnarToRow [codegen id : 1] +(16) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(17) CometHashAggregate 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#13 AS item_sk#18, count(1)#17 AS cnt#19] -(20) Filter [codegen id : 2] -Input [2]: [item_sk#18, cnt#19] -Condition : (cnt#19 > 4) +(18) CometFilter +Input [2]: [item_sk#17, cnt#18] +Condition : (cnt#18 > 4) -(21) Project [codegen id : 2] -Output [1]: [item_sk#18] -Input [2]: [item_sk#18, cnt#19] +(19) CometProject +Input [2]: [item_sk#17, cnt#18] +Arguments: [item_sk#17], [item_sk#17] -(22) BroadcastExchange -Input [1]: [item_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(20) CometBroadcastExchange +Input [1]: [item_sk#17] +Arguments: [item_sk#17] -(23) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#2] -Right keys [1]: [item_sk#18] -Join type: LeftSemi -Join condition: None +(21) CometBroadcastHashJoin +Left output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Right output [1]: [item_sk#17] +Arguments: [cs_item_sk#2], [item_sk#17], LeftSemi, BuildRight -(24) Project [codegen id : 3] -Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +(22) CometProject Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(25) Exchange +(23) CometColumnarExchange 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=3] +Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(26) Sort [codegen id : 4] +(24) CometSort 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 +Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1 ASC NULLS FIRST] -(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] +(25) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(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) +(26) CometFilter +Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] +Condition : isnotnull(ss_customer_sk#19) -(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] +(27) CometProject +Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] +Arguments: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21], [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] -(30) Scan parquet spark_catalog.default.customer -Output [1]: [c_customer_sk#24] +(28) Scan parquet spark_catalog.default.customer +Output [1]: [c_customer_sk#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(31) CometFilter -Input [1]: [c_customer_sk#24] -Condition : isnotnull(c_customer_sk#24) +(29) CometFilter +Input [1]: [c_customer_sk#23] +Condition : isnotnull(c_customer_sk#23) + +(30) CometBroadcastExchange +Input [1]: [c_customer_sk#23] +Arguments: [c_customer_sk#23] -(32) CometBroadcastExchange -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24] +(31) CometBroadcastHashJoin +Left output [3]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] +Right output [1]: [c_customer_sk#23] +Arguments: [ss_customer_sk#19], [c_customer_sk#23], Inner, BuildRight -(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 +(32) CometProject +Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, c_customer_sk#23] +Arguments: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23], [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] -(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] +(33) CometHashAggregate +Input [3]: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] +Keys [1]: [c_customer_sk#23] +Functions [1]: [partial_sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] + +(34) CometColumnarExchange +Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] +Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (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))] - -(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] - -(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))#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 : 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 : 6] -Output [1]: [c_customer_sk#24] -Input [2]: [c_customer_sk#24, ssales#28] - -(41) Sort [codegen id : 6] -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 - -(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 - -(43) Project [codegen id : 8] -Output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] +Keys [1]: [c_customer_sk#23] +Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] + +(36) CometFilter +Input [2]: [c_customer_sk#23, ssales#26] +Condition : (isnotnull(ssales#26) AND (cast(ssales#26 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#27, [id=#28]))) + +(37) CometProject +Input [2]: [c_customer_sk#23, ssales#26] +Arguments: [c_customer_sk#23], [c_customer_sk#23] + +(38) CometSort +Input [1]: [c_customer_sk#23] +Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] + +(39) CometSortMergeJoin +Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Right output [1]: [c_customer_sk#23] +Arguments: [cs_bill_customer_sk#1], [c_customer_sk#23], LeftSemi + +(40) CometProject Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Arguments: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] + +(41) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#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,2), IsNotNull(d_date_sk)] +ReadSchema: struct + +(42) CometFilter +Input [3]: [d_date_sk#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 = 2)) AND isnotnull(d_date_sk#29)) + +(43) CometProject +Input [3]: [d_date_sk#29, d_year#30, d_moy#31] +Arguments: [d_date_sk#29], [d_date_sk#29] -(44) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#31] +(44) CometBroadcastExchange +Input [1]: [d_date_sk#29] +Arguments: [d_date_sk#29] -(45) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#31] -Join type: Inner -Join condition: None +(45) CometBroadcastHashJoin +Left output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Right output [1]: [d_date_sk#29] +Arguments: [cs_sold_date_sk#5], [d_date_sk#29], Inner, BuildRight -(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) CometProject +Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#29] +Arguments: [sales#32], [(cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4) AS sales#32] (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] @@ -291,94 +291,87 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#37), dynamicpruningexpression(ws_sold_date_sk#37 IN dynamicpruning#38)] ReadSchema: struct -(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] - -(49) ReusedExchange [Reuses operator id: 22] -Output [1]: [item_sk#18] +(48) ReusedExchange [Reuses operator id: 20] +Output [1]: [item_sk#17] -(50) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ws_item_sk#33] -Right keys [1]: [item_sk#18] -Join type: LeftSemi -Join condition: None +(49) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#33, ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] +Right output [1]: [item_sk#17] +Arguments: [ws_item_sk#33], [item_sk#17], LeftSemi, BuildRight -(51) Project [codegen id : 11] -Output [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] +(50) CometProject Input [5]: [ws_item_sk#33, ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] +Arguments: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37], [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -(52) Exchange +(51) CometColumnarExchange 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] +Arguments: hashpartitioning(ws_bill_customer_sk#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(53) Sort [codegen id : 12] +(52) CometSort 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 - -(54) ReusedExchange [Reuses operator id: 37] -Output [3]: [c_customer_sk#24, sum#25, isEmpty#26] - -(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))#27] -Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#27 AS ssales#28] - -(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]))) - -(57) Project [codegen id : 14] -Output [1]: [c_customer_sk#24] -Input [2]: [c_customer_sk#24, ssales#28] - -(58) Sort [codegen id : 14] -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 - -(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 - -(60) Project [codegen id : 16] -Output [3]: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] +Arguments: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37], [ws_bill_customer_sk#34 ASC NULLS FIRST] + +(53) ReusedExchange [Reuses operator id: 34] +Output [3]: [c_customer_sk#23, sum#24, isEmpty#25] + +(54) CometHashAggregate +Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] +Keys [1]: [c_customer_sk#23] +Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] + +(55) CometFilter +Input [2]: [c_customer_sk#23, ssales#26] +Condition : (isnotnull(ssales#26) AND (cast(ssales#26 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#27, [id=#28]))) + +(56) CometProject +Input [2]: [c_customer_sk#23, ssales#26] +Arguments: [c_customer_sk#23], [c_customer_sk#23] + +(57) CometSort +Input [1]: [c_customer_sk#23] +Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] + +(58) CometSortMergeJoin +Left output [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] +Right output [1]: [c_customer_sk#23] +Arguments: [ws_bill_customer_sk#34], [c_customer_sk#23], LeftSemi + +(59) CometProject Input [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] +Arguments: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37], [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -(61) ReusedExchange [Reuses operator id: 72] +(60) ReusedExchange [Reuses operator id: 44] Output [1]: [d_date_sk#39] -(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 +(61) CometBroadcastHashJoin +Left output [3]: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] +Right output [1]: [d_date_sk#39] +Arguments: [ws_sold_date_sk#37], [d_date_sk#39], Inner, BuildRight -(63) Project [codegen id : 16] -Output [1]: [(cast(ws_quantity#35 as decimal(10,0)) * ws_list_price#36) AS sales#40] +(62) CometProject Input [4]: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37, d_date_sk#39] +Arguments: [sales#40], [(cast(ws_quantity#35 as decimal(10,0)) * ws_list_price#36) AS sales#40] -(64) Union +(63) CometUnion +Child 0 Input [1]: [sales#32] +Child 1 Input [1]: [sales#40] -(65) HashAggregate [codegen id : 17] +(64) CometHashAggregate Input [1]: [sales#32] Keys: [] Functions [1]: [partial_sum(sales#32)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [2]: [sum#43, isEmpty#44] -(66) Exchange -Input [2]: [sum#43, isEmpty#44] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +(65) CometColumnarExchange +Input [2]: [sum#41, isEmpty#42] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(67) HashAggregate [codegen id : 18] -Input [2]: [sum#43, isEmpty#44] +(66) CometHashAggregate +Input [2]: [sum#41, isEmpty#42] Keys: [] Functions [1]: [sum(sales#32)] -Aggregate Attributes [1]: [sum(sales#32)#45] -Results [1]: [sum(sales#32)#45 AS sum(sales)#46] + +(67) ColumnarToRow [codegen id : 1] +Input [1]: [sum(sales)#43] ===== Subqueries ===== @@ -391,28 +384,28 @@ BroadcastExchange (72) (68) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#31, d_year#47, d_moy#48] +Output [3]: [d_date_sk#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,2), IsNotNull(d_date_sk)] ReadSchema: struct (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)) +Input [3]: [d_date_sk#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 = 2)) AND isnotnull(d_date_sk#29)) (70) CometProject -Input [3]: [d_date_sk#31, d_year#47, d_moy#48] -Arguments: [d_date_sk#31], [d_date_sk#31] +Input [3]: [d_date_sk#29, d_year#30, d_moy#31] +Arguments: [d_date_sk#29], [d_date_sk#29] (71) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#31] +Input [1]: [d_date_sk#29] (72) BroadcastExchange -Input [1]: [d_date_sk#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Input [1]: [d_date_sk#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 2 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (77) +- * ColumnarToRow (76) +- CometProject (75) @@ -440,15 +433,15 @@ Input [2]: [d_date_sk#10, d_date#11] (77) BroadcastExchange Input [2]: [d_date_sk#10, d_date#11] -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) +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] + +Subquery:3 Hosting operator id = 36 Hosting Expression = Subquery scalar-subquery#27, [id=#28] +* ColumnarToRow (95) ++- CometHashAggregate (94) + +- CometColumnarExchange (93) + +- CometHashAggregate (92) + +- CometHashAggregate (91) + +- CometColumnarExchange (90) +- CometHashAggregate (89) +- CometProject (88) +- CometBroadcastHashJoin (87) @@ -464,95 +457,89 @@ Subquery:3 Hosting operator id = 39 Hosting Expression = Subquery scalar-subquer (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] +Output [4]: [ss_customer_sk#44, ss_quantity#45, ss_sales_price#46, ss_sold_date_sk#47] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#52), dynamicpruningexpression(ss_sold_date_sk#52 IN dynamicpruning#53)] +PartitionFilters: [isnotnull(ss_sold_date_sk#47), dynamicpruningexpression(ss_sold_date_sk#47 IN dynamicpruning#48)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (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) +Input [4]: [ss_customer_sk#44, ss_quantity#45, ss_sales_price#46, ss_sold_date_sk#47] +Condition : isnotnull(ss_customer_sk#44) -(80) ReusedExchange [Reuses operator id: 32] -Output [1]: [c_customer_sk#54] +(80) ReusedExchange [Reuses operator id: 30] +Output [1]: [c_customer_sk#49] (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 +Left output [4]: [ss_customer_sk#44, ss_quantity#45, ss_sales_price#46, ss_sold_date_sk#47] +Right output [1]: [c_customer_sk#49] +Arguments: [ss_customer_sk#44], [c_customer_sk#49], Inner, BuildRight (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] +Input [5]: [ss_customer_sk#44, ss_quantity#45, ss_sales_price#46, ss_sold_date_sk#47, c_customer_sk#49] +Arguments: [ss_quantity#45, ss_sales_price#46, ss_sold_date_sk#47, c_customer_sk#49], [ss_quantity#45, ss_sales_price#46, ss_sold_date_sk#47, c_customer_sk#49] (83) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#55, d_year#56] +Output [2]: [d_date_sk#50, d_year#51] 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) 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)) +Input [2]: [d_date_sk#50, d_year#51] +Condition : (d_year#51 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#50)) (85) CometProject -Input [2]: [d_date_sk#55, d_year#56] -Arguments: [d_date_sk#55], [d_date_sk#55] +Input [2]: [d_date_sk#50, d_year#51] +Arguments: [d_date_sk#50], [d_date_sk#50] (86) CometBroadcastExchange -Input [1]: [d_date_sk#55] -Arguments: [d_date_sk#55] +Input [1]: [d_date_sk#50] +Arguments: [d_date_sk#50] (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 +Left output [4]: [ss_quantity#45, ss_sales_price#46, ss_sold_date_sk#47, c_customer_sk#49] +Right output [1]: [d_date_sk#50] +Arguments: [ss_sold_date_sk#47], [d_date_sk#50], 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] +Input [5]: [ss_quantity#45, ss_sales_price#46, ss_sold_date_sk#47, c_customer_sk#49, d_date_sk#50] +Arguments: [ss_quantity#45, ss_sales_price#46, c_customer_sk#49], [ss_quantity#45, ss_sales_price#46, c_customer_sk#49] (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] - -(93) HashAggregate [codegen id : 2] -Input [1]: [csales#60] +Input [3]: [ss_quantity#45, ss_sales_price#46, c_customer_sk#49] +Keys [1]: [c_customer_sk#49] +Functions [1]: [partial_sum((cast(ss_quantity#45 as decimal(10,0)) * ss_sales_price#46))] + +(90) CometColumnarExchange +Input [3]: [c_customer_sk#49, sum#52, isEmpty#53] +Arguments: hashpartitioning(c_customer_sk#49, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(91) CometHashAggregate +Input [3]: [c_customer_sk#49, sum#52, isEmpty#53] +Keys [1]: [c_customer_sk#49] +Functions [1]: [sum((cast(ss_quantity#45 as decimal(10,0)) * ss_sales_price#46))] + +(92) CometHashAggregate +Input [1]: [csales#54] Keys: [] -Functions [1]: [partial_max(csales#60)] -Aggregate Attributes [1]: [max#61] -Results [1]: [max#62] +Functions [1]: [partial_max(csales#54)] -(94) Exchange -Input [1]: [max#62] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] +(93) CometColumnarExchange +Input [1]: [max#55] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(95) HashAggregate [codegen id : 3] -Input [1]: [max#62] +(94) CometHashAggregate +Input [1]: [max#55] Keys: [] -Functions [1]: [max(csales#60)] -Aggregate Attributes [1]: [max(csales#60)#63] -Results [1]: [max(csales#60)#63 AS tpcds_cmax#64] +Functions [1]: [max(csales#54)] + +(95) ColumnarToRow [codegen id : 1] +Input [1]: [tpcds_cmax#56] -Subquery:4 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#52 IN dynamicpruning#53 +Subquery:4 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#47 IN dynamicpruning#48 BroadcastExchange (100) +- * ColumnarToRow (99) +- CometProject (98) @@ -561,29 +548,29 @@ BroadcastExchange (100) (96) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#55, d_year#56] +Output [2]: [d_date_sk#50, d_year#51] 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 (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)) +Input [2]: [d_date_sk#50, d_year#51] +Condition : (d_year#51 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#50)) (98) CometProject -Input [2]: [d_date_sk#55, d_year#56] -Arguments: [d_date_sk#55], [d_date_sk#55] +Input [2]: [d_date_sk#50, d_year#51] +Arguments: [d_date_sk#50], [d_date_sk#50] (99) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#55] +Input [1]: [d_date_sk#50] (100) BroadcastExchange -Input [1]: [d_date_sk#55] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] +Input [1]: [d_date_sk#50] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] Subquery:5 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#37 IN dynamicpruning#6 -Subquery:6 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquery scalar-subquery#29, [id=#30] +Subquery:6 Hosting operator id = 55 Hosting Expression = ReusedSubquery Subquery scalar-subquery#27, [id=#28] 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 e065a0a87..18cf2b2c6 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,150 +1,116 @@ -WholeStageCodegen (18) - HashAggregate [sum,isEmpty] [sum(sales),sum(sales),sum,isEmpty] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (17) - HashAggregate [sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - 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 (4) - Sort [cs_bill_customer_sk] - InputAdapter - Exchange [cs_bill_customer_sk] #2 - WholeStageCodegen (3) - Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - BroadcastHashJoin [cs_item_sk,item_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #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 + CometHashAggregate [sum(sales),sum,isEmpty,sum(sales)] + CometColumnarExchange #1 + CometHashAggregate [sum,isEmpty,sales] + CometUnion [sales] + CometProject [cs_quantity,cs_list_price] [sales] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] + CometProject [cs_quantity,cs_list_price,cs_sold_date_sk] + CometSortMergeJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk] + CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + CometColumnarExchange [cs_bill_customer_sk] #2 + CometProject [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [item_sk] #4 + CometProject [item_sk] + CometFilter [item_sk,cnt] + CometHashAggregate [item_sk,cnt,_groupingexpression,i_item_sk,d_date,count,count(1)] + CometColumnarExchange [_groupingexpression,i_item_sk,d_date] #5 + CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] + CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] + CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] + CometProject [ss_item_sk,d_date] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #6 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] - CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] - CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [i_item_sk,i_item_desc] #8 - CometFilter [i_item_sk,i_item_desc] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - InputAdapter - WholeStageCodegen (6) - Sort [c_customer_sk] - Project [c_customer_sk] - Filter [ssales] - Subquery #3 - WholeStageCodegen (3) - HashAggregate [max] [max(csales),tpcds_cmax,max] - InputAdapter - 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] #12 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] - CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] - CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_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 #13 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange [d_date_sk] #14 + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + CometBroadcastExchange [d_date_sk,d_date] #7 + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + CometBroadcastExchange [i_item_sk,i_item_desc] #8 + CometFilter [i_item_sk,i_item_desc] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + CometSort [c_customer_sk] + CometProject [c_customer_sk] + CometFilter [c_customer_sk,ssales] + Subquery #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [tpcds_cmax,max,max(csales)] + CometColumnarExchange #11 + CometHashAggregate [max,csales] + CometHashAggregate [csales,c_customer_sk,sum,isEmpty,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + CometColumnarExchange [c_customer_sk] #12 + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] + CometProject [ss_quantity,ss_sales_price,c_customer_sk] + CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] + CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_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 #13 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] 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] #9 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] - CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] - CometProject [ss_customer_sk,ss_quantity,ss_sales_price] - CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [c_customer_sk] #10 - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk] - InputAdapter - ReusedExchange [d_date_sk] #3 - 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 (12) - Sort [ws_bill_customer_sk] - InputAdapter - 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 - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [item_sk] #4 - InputAdapter - 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] #9 - InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [c_customer_sk] #10 + CometBroadcastExchange [d_date_sk] #14 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometHashAggregate [c_customer_sk,ssales,sum,isEmpty,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + CometColumnarExchange [c_customer_sk] #9 + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] + CometProject [ss_quantity,ss_sales_price,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] + CometProject [ss_customer_sk,ss_quantity,ss_sales_price] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometBroadcastExchange [c_customer_sk] #10 + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk] + CometBroadcastExchange [d_date_sk] #15 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [ws_quantity,ws_list_price] [sales] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] + CometProject [ws_quantity,ws_list_price,ws_sold_date_sk] + CometSortMergeJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk] + CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + CometColumnarExchange [ws_bill_customer_sk] #16 + CometProject [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [item_sk] #4 + CometSort [c_customer_sk] + CometProject [c_customer_sk] + CometFilter [c_customer_sk,ssales] + ReusedSubquery [tpcds_cmax] #3 + CometHashAggregate [c_customer_sk,ssales,sum,isEmpty,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + ReusedExchange [c_customer_sk,sum,isEmpty] #9 + ReusedExchange [d_date_sk] #15 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 beb975753..0f36e0d18 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,92 +1,91 @@ == Physical Plan == -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 (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) +* ColumnarToRow (87) ++- CometTakeOrderedAndProject (86) + +- CometUnion (85) + :- CometHashAggregate (62) + : +- CometColumnarExchange (61) + : +- CometHashAggregate (60) + : +- CometProject (59) + : +- CometBroadcastHashJoin (58) + : :- CometProject (53) + : : +- CometBroadcastHashJoin (52) + : : :- CometSortMergeJoin (40) + : : : :- CometSort (25) + : : : : +- CometColumnarExchange (24) + : : : : +- CometProject (23) + : : : : +- CometBroadcastHashJoin (22) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : +- CometBroadcastExchange (21) + : : : : +- CometProject (20) + : : : : +- CometFilter (19) + : : : : +- CometHashAggregate (18) + : : : : +- CometColumnarExchange (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) + : : : +- CometSort (39) + : : : +- CometProject (38) + : : : +- CometFilter (37) + : : : +- CometHashAggregate (36) + : : : +- CometColumnarExchange (35) + : : : +- CometHashAggregate (34) + : : : +- CometProject (33) + : : : +- CometBroadcastHashJoin (32) + : : : :- CometProject (28) + : : : : +- CometFilter (27) + : : : : +- CometScan parquet spark_catalog.default.store_sales (26) + : : : +- CometBroadcastExchange (31) + : : : +- CometFilter (30) + : : : +- CometScan parquet spark_catalog.default.customer (29) + : : +- CometBroadcastExchange (51) + : : +- CometSortMergeJoin (50) + : : :- CometSort (44) + : : : +- CometColumnarExchange (43) + : : : +- CometFilter (42) + : : : +- CometScan parquet spark_catalog.default.customer (41) + : : +- CometSort (49) + : : +- CometProject (48) + : : +- CometFilter (47) + : : +- CometHashAggregate (46) + : : +- ReusedExchange (45) + : +- CometBroadcastExchange (57) + : +- CometProject (56) + : +- CometFilter (55) + : +- CometScan parquet spark_catalog.default.date_dim (54) + +- CometHashAggregate (84) + +- CometColumnarExchange (83) + +- CometHashAggregate (82) + +- CometProject (81) + +- CometBroadcastHashJoin (80) + :- CometProject (78) + : +- CometBroadcastHashJoin (77) + : :- CometSortMergeJoin (75) + : : :- CometSort (69) + : : : +- CometColumnarExchange (68) + : : : +- CometProject (67) + : : : +- CometBroadcastHashJoin (66) + : : : :- CometFilter (64) + : : : : +- CometScan parquet spark_catalog.default.web_sales (63) + : : : +- ReusedExchange (65) + : : +- CometSort (74) + : : +- CometProject (73) + : : +- CometFilter (72) + : : +- CometHashAggregate (71) + : : +- ReusedExchange (70) + : +- ReusedExchange (76) + +- ReusedExchange (79) (1) Scan parquet spark_catalog.default.catalog_sales @@ -101,10 +100,7 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] Condition : isnotnull(ss_item_sk#7) -(6) Scan parquet spark_catalog.default.date_dim +(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 -(7) CometFilter +(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)) -(8) CometProject +(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] -(9) CometBroadcastExchange +(8) CometBroadcastExchange Input [2]: [d_date_sk#10, d_date#11] Arguments: [d_date_sk#10, d_date#11] -(10) CometBroadcastHashJoin +(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 -(11) CometProject +(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] -(12) Scan parquet spark_catalog.default.item +(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 -(13) CometFilter +(12) CometFilter Input [2]: [i_item_sk#13, i_item_desc#14] Condition : isnotnull(i_item_sk#13) -(14) CometBroadcastExchange +(13) CometBroadcastExchange Input [2]: [i_item_sk#13, i_item_desc#14] Arguments: [i_item_sk#13, i_item_desc#14] -(15) CometBroadcastHashJoin +(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 -(16) CometProject +(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] -(17) CometHashAggregate +(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)] -(18) ColumnarToRow [codegen id : 1] -Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] - -(19) Exchange +(17) CometColumnarExchange 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] +Arguments: hashpartitioning(_groupingexpression#15, i_item_sk#13, d_date#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(20) HashAggregate [codegen id : 2] +(18) CometHashAggregate 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#13 AS item_sk#18, count(1)#17 AS cnt#19] -(21) Filter [codegen id : 2] -Input [2]: [item_sk#18, cnt#19] -Condition : (cnt#19 > 4) +(19) CometFilter +Input [2]: [item_sk#17, cnt#18] +Condition : (cnt#18 > 4) -(22) Project [codegen id : 2] -Output [1]: [item_sk#18] -Input [2]: [item_sk#18, cnt#19] +(20) CometProject +Input [2]: [item_sk#17, cnt#18] +Arguments: [item_sk#17], [item_sk#17] -(23) BroadcastExchange -Input [1]: [item_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(21) CometBroadcastExchange +Input [1]: [item_sk#17] +Arguments: [item_sk#17] -(24) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#2] -Right keys [1]: [item_sk#18] -Join type: LeftSemi -Join condition: None +(22) CometBroadcastHashJoin +Left output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Right output [1]: [item_sk#17] +Arguments: [cs_item_sk#2], [item_sk#17], LeftSemi, BuildRight -(25) Project [codegen id : 3] -Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +(23) CometProject Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(26) Exchange +(24) CometColumnarExchange 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=3] +Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(27) Sort [codegen id : 4] +(25) CometSort 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 +Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1 ASC NULLS FIRST] -(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] +(26) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(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) +(27) CometFilter +Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] +Condition : isnotnull(ss_customer_sk#19) -(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] +(28) CometProject +Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] +Arguments: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21], [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] -(31) Scan parquet spark_catalog.default.customer -Output [1]: [c_customer_sk#24] +(29) Scan parquet spark_catalog.default.customer +Output [1]: [c_customer_sk#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(32) CometFilter -Input [1]: [c_customer_sk#24] -Condition : isnotnull(c_customer_sk#24) +(30) CometFilter +Input [1]: [c_customer_sk#23] +Condition : isnotnull(c_customer_sk#23) -(33) CometBroadcastExchange -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24] +(31) CometBroadcastExchange +Input [1]: [c_customer_sk#23] +Arguments: [c_customer_sk#23] -(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 +(32) CometBroadcastHashJoin +Left output [3]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] +Right output [1]: [c_customer_sk#23] +Arguments: [ss_customer_sk#19], [c_customer_sk#23], Inner, BuildRight -(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] +(33) CometProject +Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, c_customer_sk#23] +Arguments: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23], [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] + +(34) CometHashAggregate +Input [3]: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] +Keys [1]: [c_customer_sk#23] +Functions [1]: [partial_sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] + +(35) CometColumnarExchange +Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] +Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (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))] - -(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] - -(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))#27] -Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#27 AS ssales#28] - -(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]))) - -(41) Project [codegen id : 6] -Output [1]: [c_customer_sk#24] -Input [2]: [c_customer_sk#24, ssales#28] - -(42) Sort [codegen id : 6] -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 - -(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 - -(44) Scan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33] +Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] +Keys [1]: [c_customer_sk#23] +Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] + +(37) CometFilter +Input [2]: [c_customer_sk#23, ssales#26] +Condition : (isnotnull(ssales#26) AND (cast(ssales#26 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#27, [id=#28]))) + +(38) CometProject +Input [2]: [c_customer_sk#23, ssales#26] +Arguments: [c_customer_sk#23], [c_customer_sk#23] + +(39) CometSort +Input [1]: [c_customer_sk#23] +Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] + +(40) CometSortMergeJoin +Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Right output [1]: [c_customer_sk#23] +Arguments: [cs_bill_customer_sk#1], [c_customer_sk#23], LeftSemi + +(41) Scan parquet spark_catalog.default.customer +Output [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(45) CometFilter -Input [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33] -Condition : isnotnull(c_customer_sk#31) - -(46) ColumnarToRow [codegen id : 7] -Input [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33] - -(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] - -(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 - -(49) ReusedExchange [Reuses operator id: 38] -Output [3]: [c_customer_sk#24, sum#25, isEmpty#26] - -(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))#27] -Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#27 AS ssales#28] - -(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]))) - -(52) Project [codegen id : 10] -Output [1]: [c_customer_sk#24] -Input [2]: [c_customer_sk#24, ssales#28] - -(53) Sort [codegen id : 10] -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 - -(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 - -(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] - -(56) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#31] -Join type: Inner -Join condition: None - -(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] - -(58) ReusedExchange [Reuses operator id: 93] -Output [1]: [d_date_sk#34] - -(59) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#34] -Join type: Inner -Join condition: None - -(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] - -(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] +(42) CometFilter +Input [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] +Condition : isnotnull(c_customer_sk#29) + +(43) CometColumnarExchange +Input [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] +Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(44) CometSort +Input [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] +Arguments: [c_customer_sk#29, c_first_name#30, c_last_name#31], [c_customer_sk#29 ASC NULLS FIRST] + +(45) ReusedExchange [Reuses operator id: 35] +Output [3]: [c_customer_sk#23, sum#24, isEmpty#25] + +(46) CometHashAggregate +Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] +Keys [1]: [c_customer_sk#23] +Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] + +(47) CometFilter +Input [2]: [c_customer_sk#23, ssales#26] +Condition : (isnotnull(ssales#26) AND (cast(ssales#26 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#27, [id=#28]))) + +(48) CometProject +Input [2]: [c_customer_sk#23, ssales#26] +Arguments: [c_customer_sk#23], [c_customer_sk#23] + +(49) CometSort +Input [1]: [c_customer_sk#23] +Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] + +(50) CometSortMergeJoin +Left output [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] +Right output [1]: [c_customer_sk#23] +Arguments: [c_customer_sk#29], [c_customer_sk#23], LeftSemi + +(51) CometBroadcastExchange +Input [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] +Arguments: [c_customer_sk#29, c_first_name#30, c_last_name#31] + +(52) CometBroadcastHashJoin +Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Right output [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] +Arguments: [cs_bill_customer_sk#1], [c_customer_sk#29], Inner, BuildRight + +(53) CometProject +Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#29, c_first_name#30, c_last_name#31] +Arguments: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#30, c_last_name#31], [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#30, c_last_name#31] + +(54) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#32, d_year#33, d_moy#34] +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 + +(55) CometFilter +Input [3]: [d_date_sk#32, d_year#33, d_moy#34] +Condition : ((((isnotnull(d_year#33) AND isnotnull(d_moy#34)) AND (d_year#33 = 2000)) AND (d_moy#34 = 2)) AND isnotnull(d_date_sk#32)) + +(56) CometProject +Input [3]: [d_date_sk#32, d_year#33, d_moy#34] +Arguments: [d_date_sk#32], [d_date_sk#32] + +(57) CometBroadcastExchange +Input [1]: [d_date_sk#32] +Arguments: [d_date_sk#32] + +(58) CometBroadcastHashJoin +Left output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#30, c_last_name#31] +Right output [1]: [d_date_sk#32] +Arguments: [cs_sold_date_sk#5], [d_date_sk#32], Inner, BuildRight + +(59) CometProject +Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#30, c_last_name#31, d_date_sk#32] +Arguments: [cs_quantity#3, cs_list_price#4, c_first_name#30, c_last_name#31], [cs_quantity#3, cs_list_price#4, c_first_name#30, c_last_name#31] + +(60) CometHashAggregate +Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#30, c_last_name#31] +Keys [2]: [c_last_name#31, c_first_name#30] Functions [1]: [partial_sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] -Aggregate Attributes [2]: [sum#35, isEmpty#36] -Results [4]: [c_last_name#33, c_first_name#32, sum#37, isEmpty#38] -(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] +(61) CometColumnarExchange +Input [4]: [c_last_name#31, c_first_name#30, sum#35, isEmpty#36] +Arguments: hashpartitioning(c_last_name#31, c_first_name#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(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] +(62) CometHashAggregate +Input [4]: [c_last_name#31, c_first_name#30, sum#35, isEmpty#36] +Keys [2]: [c_last_name#31, c_first_name#30] 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))#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] -(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] +(63) Scan parquet spark_catalog.default.web_sales +Output [5]: [ws_item_sk#37, ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#45), dynamicpruningexpression(ws_sold_date_sk#45 IN dynamicpruning#46)] +PartitionFilters: [isnotnull(ws_sold_date_sk#41), dynamicpruningexpression(ws_sold_date_sk#41 IN dynamicpruning#42)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(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) - -(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] - -(67) ReusedExchange [Reuses operator id: 23] -Output [1]: [item_sk#18] - -(68) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#41] -Right keys [1]: [item_sk#18] -Join type: LeftSemi -Join condition: None - -(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] - -(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] - -(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 - -(72) ReusedExchange [Reuses operator id: 38] -Output [3]: [c_customer_sk#24, sum#25, isEmpty#26] - -(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))#27] -Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#27 AS ssales#28] - -(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]))) - -(75) Project [codegen id : 20] -Output [1]: [c_customer_sk#24] -Input [2]: [c_customer_sk#24, ssales#28] - -(76) Sort [codegen id : 20] -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 - -(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 - -(78) ReusedExchange [Reuses operator id: 55] -Output [3]: [c_customer_sk#47, c_first_name#48, c_last_name#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 - -(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] - -(81) ReusedExchange [Reuses operator id: 93] -Output [1]: [d_date_sk#50] - -(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 - -(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] - -(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] - -(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] - -(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] - -(87) Union - -(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] +(64) CometFilter +Input [5]: [ws_item_sk#37, ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] +Condition : isnotnull(ws_bill_customer_sk#38) + +(65) ReusedExchange [Reuses operator id: 21] +Output [1]: [item_sk#17] + +(66) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#37, ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] +Right output [1]: [item_sk#17] +Arguments: [ws_item_sk#37], [item_sk#17], LeftSemi, BuildRight + +(67) CometProject +Input [5]: [ws_item_sk#37, ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] +Arguments: [ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41], [ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] + +(68) CometColumnarExchange +Input [4]: [ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] +Arguments: hashpartitioning(ws_bill_customer_sk#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(69) CometSort +Input [4]: [ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] +Arguments: [ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41], [ws_bill_customer_sk#38 ASC NULLS FIRST] + +(70) ReusedExchange [Reuses operator id: 35] +Output [3]: [c_customer_sk#23, sum#24, isEmpty#25] + +(71) CometHashAggregate +Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] +Keys [1]: [c_customer_sk#23] +Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] + +(72) CometFilter +Input [2]: [c_customer_sk#23, ssales#26] +Condition : (isnotnull(ssales#26) AND (cast(ssales#26 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#27, [id=#28]))) + +(73) CometProject +Input [2]: [c_customer_sk#23, ssales#26] +Arguments: [c_customer_sk#23], [c_customer_sk#23] + +(74) CometSort +Input [1]: [c_customer_sk#23] +Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] + +(75) CometSortMergeJoin +Left output [4]: [ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] +Right output [1]: [c_customer_sk#23] +Arguments: [ws_bill_customer_sk#38], [c_customer_sk#23], LeftSemi + +(76) ReusedExchange [Reuses operator id: 51] +Output [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] + +(77) CometBroadcastHashJoin +Left output [4]: [ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] +Right output [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] +Arguments: [ws_bill_customer_sk#38], [c_customer_sk#43], Inner, BuildRight + +(78) CometProject +Input [7]: [ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41, c_customer_sk#43, c_first_name#44, c_last_name#45] +Arguments: [ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41, c_first_name#44, c_last_name#45], [ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41, c_first_name#44, c_last_name#45] + +(79) ReusedExchange [Reuses operator id: 57] +Output [1]: [d_date_sk#46] + +(80) CometBroadcastHashJoin +Left output [5]: [ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41, c_first_name#44, c_last_name#45] +Right output [1]: [d_date_sk#46] +Arguments: [ws_sold_date_sk#41], [d_date_sk#46], Inner, BuildRight + +(81) CometProject +Input [6]: [ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41, c_first_name#44, c_last_name#45, d_date_sk#46] +Arguments: [ws_quantity#39, ws_list_price#40, c_first_name#44, c_last_name#45], [ws_quantity#39, ws_list_price#40, c_first_name#44, c_last_name#45] + +(82) CometHashAggregate +Input [4]: [ws_quantity#39, ws_list_price#40, c_first_name#44, c_last_name#45] +Keys [2]: [c_last_name#45, c_first_name#44] +Functions [1]: [partial_sum((cast(ws_quantity#39 as decimal(10,0)) * ws_list_price#40))] + +(83) CometColumnarExchange +Input [4]: [c_last_name#45, c_first_name#44, sum#47, isEmpty#48] +Arguments: hashpartitioning(c_last_name#45, c_first_name#44, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(84) CometHashAggregate +Input [4]: [c_last_name#45, c_first_name#44, sum#47, isEmpty#48] +Keys [2]: [c_last_name#45, c_first_name#44] +Functions [1]: [sum((cast(ws_quantity#39 as decimal(10,0)) * ws_list_price#40))] + +(85) CometUnion +Child 0 Input [3]: [c_last_name#31, c_first_name#30, sales#49] +Child 1 Input [3]: [c_last_name#45, c_first_name#44, sales#50] + +(86) CometTakeOrderedAndProject +Input [3]: [c_last_name#31, c_first_name#30, sales#49] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#31 ASC NULLS FIRST,c_first_name#30 ASC NULLS FIRST,sales#49 ASC NULLS FIRST], output=[c_last_name#31,c_first_name#30,sales#49]), [c_last_name#31, c_first_name#30, sales#49], 100, [c_last_name#31 ASC NULLS FIRST, c_first_name#30 ASC NULLS FIRST, sales#49 ASC NULLS FIRST], [c_last_name#31, c_first_name#30, sales#49] + +(87) ColumnarToRow [codegen id : 1] +Input [3]: [c_last_name#31, c_first_name#30, sales#49] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (93) -+- * ColumnarToRow (92) - +- CometProject (91) - +- CometFilter (90) - +- CometScan parquet spark_catalog.default.date_dim (89) +BroadcastExchange (92) ++- * ColumnarToRow (91) + +- CometProject (90) + +- CometFilter (89) + +- CometScan parquet spark_catalog.default.date_dim (88) -(89) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#34, d_year#57, d_moy#58] +(88) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#32, d_year#33, d_moy#34] 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 -(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)) +(89) CometFilter +Input [3]: [d_date_sk#32, d_year#33, d_moy#34] +Condition : ((((isnotnull(d_year#33) AND isnotnull(d_moy#34)) AND (d_year#33 = 2000)) AND (d_moy#34 = 2)) AND isnotnull(d_date_sk#32)) -(91) CometProject -Input [3]: [d_date_sk#34, d_year#57, d_moy#58] -Arguments: [d_date_sk#34], [d_date_sk#34] +(90) CometProject +Input [3]: [d_date_sk#32, d_year#33, d_moy#34] +Arguments: [d_date_sk#32], [d_date_sk#32] -(92) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#34] +(91) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#32] -(93) BroadcastExchange -Input [1]: [d_date_sk#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] +(92) BroadcastExchange +Input [1]: [d_date_sk#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (98) -+- * ColumnarToRow (97) - +- CometProject (96) - +- CometFilter (95) - +- CometScan parquet spark_catalog.default.date_dim (94) +Subquery:2 Hosting operator id = 3 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) -(94) Scan parquet spark_catalog.default.date_dim +(93) 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 -(95) CometFilter +(94) 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)) -(96) CometProject +(95) 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] -(97) ColumnarToRow [codegen id : 1] +(96) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_date#11] -(98) BroadcastExchange +(97) BroadcastExchange Input [2]: [d_date_sk#10, d_date#11] -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] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] + +Subquery:3 Hosting operator id = 37 Hosting Expression = Subquery scalar-subquery#27, [id=#28] +* ColumnarToRow (115) ++- CometHashAggregate (114) + +- CometColumnarExchange (113) + +- CometHashAggregate (112) + +- CometHashAggregate (111) + +- CometColumnarExchange (110) + +- CometHashAggregate (109) + +- CometProject (108) + +- CometBroadcastHashJoin (107) + :- CometProject (102) + : +- CometBroadcastHashJoin (101) + : :- CometFilter (99) + : : +- CometScan parquet spark_catalog.default.store_sales (98) + : +- ReusedExchange (100) + +- CometBroadcastExchange (106) + +- CometProject (105) + +- CometFilter (104) + +- CometScan parquet spark_catalog.default.date_dim (103) + + +(98) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54] 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#54), dynamicpruningexpression(ss_sold_date_sk#54 IN dynamicpruning#55)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(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) +(99) CometFilter +Input [4]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54] +Condition : isnotnull(ss_customer_sk#51) -(101) ReusedExchange [Reuses operator id: 33] -Output [1]: [c_customer_sk#64] +(100) ReusedExchange [Reuses operator id: 31] +Output [1]: [c_customer_sk#56] -(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 +(101) CometBroadcastHashJoin +Left output [4]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54] +Right output [1]: [c_customer_sk#56] +Arguments: [ss_customer_sk#51], [c_customer_sk#56], Inner, BuildRight -(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] +(102) CometProject +Input [5]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56] +Arguments: [ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56], [ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56] -(104) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#65, d_year#66] +(103) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#57, d_year#58] 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) 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] +(104) CometFilter +Input [2]: [d_date_sk#57, d_year#58] +Condition : (d_year#58 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#57)) -(107) CometBroadcastExchange -Input [1]: [d_date_sk#65] -Arguments: [d_date_sk#65] +(105) CometProject +Input [2]: [d_date_sk#57, d_year#58] +Arguments: [d_date_sk#57], [d_date_sk#57] -(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 +(106) CometBroadcastExchange +Input [1]: [d_date_sk#57] +Arguments: [d_date_sk#57] -(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] +(107) CometBroadcastHashJoin +Left output [4]: [ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56] +Right output [1]: [d_date_sk#57] +Arguments: [ss_sold_date_sk#54], [d_date_sk#57], Inner, BuildRight -(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))] +(108) CometProject +Input [5]: [ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56, d_date_sk#57] +Arguments: [ss_quantity#52, ss_sales_price#53, c_customer_sk#56], [ss_quantity#52, ss_sales_price#53, c_customer_sk#56] -(111) ColumnarToRow [codegen id : 1] -Input [3]: [c_customer_sk#64, sum#67, isEmpty#68] +(109) CometHashAggregate +Input [3]: [ss_quantity#52, ss_sales_price#53, c_customer_sk#56] +Keys [1]: [c_customer_sk#56] +Functions [1]: [partial_sum((cast(ss_quantity#52 as decimal(10,0)) * ss_sales_price#53))] -(112) Exchange -Input [3]: [c_customer_sk#64, sum#67, isEmpty#68] -Arguments: hashpartitioning(c_customer_sk#64, 5), ENSURE_REQUIREMENTS, [plan_id=12] +(110) CometColumnarExchange +Input [3]: [c_customer_sk#56, sum#59, isEmpty#60] +Arguments: hashpartitioning(c_customer_sk#56, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(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] +(111) CometHashAggregate +Input [3]: [c_customer_sk#56, sum#59, isEmpty#60] +Keys [1]: [c_customer_sk#56] +Functions [1]: [sum((cast(ss_quantity#52 as decimal(10,0)) * ss_sales_price#53))] -(114) HashAggregate [codegen id : 2] -Input [1]: [csales#70] +(112) CometHashAggregate +Input [1]: [csales#61] Keys: [] -Functions [1]: [partial_max(csales#70)] -Aggregate Attributes [1]: [max#71] -Results [1]: [max#72] +Functions [1]: [partial_max(csales#61)] -(115) Exchange -Input [1]: [max#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] +(113) CometColumnarExchange +Input [1]: [max#62] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(116) HashAggregate [codegen id : 3] -Input [1]: [max#72] +(114) CometHashAggregate +Input [1]: [max#62] Keys: [] -Functions [1]: [max(csales#70)] -Aggregate Attributes [1]: [max(csales#70)#73] -Results [1]: [max(csales#70)#73 AS tpcds_cmax#74] +Functions [1]: [max(csales#61)] + +(115) ColumnarToRow [codegen id : 1] +Input [1]: [tpcds_cmax#63] -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) +Subquery:4 Hosting operator id = 98 Hosting Expression = ss_sold_date_sk#54 IN dynamicpruning#55 +BroadcastExchange (120) ++- * ColumnarToRow (119) + +- CometProject (118) + +- CometFilter (117) + +- CometScan parquet spark_catalog.default.date_dim (116) -(117) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#65, d_year#66] +(116) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#57, d_year#58] 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 -(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)) +(117) CometFilter +Input [2]: [d_date_sk#57, d_year#58] +Condition : (d_year#58 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#57)) -(119) CometProject -Input [2]: [d_date_sk#65, d_year#66] -Arguments: [d_date_sk#65], [d_date_sk#65] +(118) CometProject +Input [2]: [d_date_sk#57, d_year#58] +Arguments: [d_date_sk#57], [d_date_sk#57] -(120) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#65] +(119) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#57] -(121) BroadcastExchange -Input [1]: [d_date_sk#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] +(120) BroadcastExchange +Input [1]: [d_date_sk#57] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -Subquery:5 Hosting operator id = 51 Hosting Expression = ReusedSubquery Subquery scalar-subquery#29, [id=#30] +Subquery:5 Hosting operator id = 47 Hosting Expression = ReusedSubquery Subquery scalar-subquery#27, [id=#28] -Subquery:6 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#45 IN dynamicpruning#6 +Subquery:6 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#41 IN dynamicpruning#6 -Subquery:7 Hosting operator id = 74 Hosting Expression = ReusedSubquery Subquery scalar-subquery#29, [id=#30] +Subquery:7 Hosting operator id = 72 Hosting Expression = ReusedSubquery Subquery scalar-subquery#27, [id=#28] 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 2862f64cc..55f17ca34 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,183 +1,137 @@ -TakeOrderedAndProject [c_last_name,c_first_name,sales] - Union - 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 (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] - Project [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - SortMergeJoin [cs_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (4) - Sort [cs_bill_customer_sk] - InputAdapter - Exchange [cs_bill_customer_sk] #2 - WholeStageCodegen (3) - Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - BroadcastHashJoin [cs_item_sk,item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] - CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] - CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [i_item_sk,i_item_desc] #8 - CometFilter [i_item_sk,i_item_desc] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - InputAdapter - WholeStageCodegen (6) - Sort [c_customer_sk] - Project [c_customer_sk] - Filter [ssales] - Subquery #3 - WholeStageCodegen (3) - HashAggregate [max] [max(csales),tpcds_cmax,max] - InputAdapter - 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] #12 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] - CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] - CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_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 #13 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - 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] #9 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] - CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] - CometProject [ss_customer_sk,ss_quantity,ss_sales_price] - CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [c_customer_sk] #10 - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk] - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (11) - SortMergeJoin [c_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (8) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #16 - WholeStageCodegen (7) +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [c_last_name,c_first_name,sales] + CometUnion [c_last_name,c_first_name,sales] + CometHashAggregate [c_last_name,c_first_name,sales,sum,isEmpty,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price))] + CometColumnarExchange [c_last_name,c_first_name] #1 + CometHashAggregate [c_last_name,c_first_name,sum,isEmpty,cs_quantity,cs_list_price] + CometProject [cs_quantity,cs_list_price,c_first_name,c_last_name] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name,d_date_sk] + CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk,c_first_name,c_last_name] + CometSortMergeJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk] + CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + CometColumnarExchange [cs_bill_customer_sk] #2 + CometProject [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,item_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - InputAdapter - 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] #9 - InputAdapter - ReusedExchange [d_date_sk] #3 - 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] #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] - Project [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - SortMergeJoin [ws_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (18) - Sort [ws_bill_customer_sk] - InputAdapter - 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 - InputAdapter - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [item_sk] #4 - InputAdapter - 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] + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [item_sk] #4 + CometProject [item_sk] + CometFilter [item_sk,cnt] + CometHashAggregate [item_sk,cnt,_groupingexpression,i_item_sk,d_date,count,count(1)] + CometColumnarExchange [_groupingexpression,i_item_sk,d_date] #5 + CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] + CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] + CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] + CometProject [ss_item_sk,d_date] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + CometBroadcastExchange [d_date_sk,d_date] #7 + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + CometBroadcastExchange [i_item_sk,i_item_desc] #8 + CometFilter [i_item_sk,i_item_desc] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + CometSort [c_customer_sk] + CometProject [c_customer_sk] + CometFilter [c_customer_sk,ssales] + Subquery #3 + WholeStageCodegen (1) + ColumnarToRow InputAdapter - ReusedExchange [c_customer_sk,sum,isEmpty] #9 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #15 - InputAdapter - ReusedExchange [d_date_sk] #3 + CometHashAggregate [tpcds_cmax,max,max(csales)] + CometColumnarExchange #11 + CometHashAggregate [max,csales] + CometHashAggregate [csales,c_customer_sk,sum,isEmpty,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + CometColumnarExchange [c_customer_sk] #12 + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] + CometProject [ss_quantity,ss_sales_price,c_customer_sk] + CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] + CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_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 #13 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [c_customer_sk] #10 + CometBroadcastExchange [d_date_sk] #14 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometHashAggregate [c_customer_sk,ssales,sum,isEmpty,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + CometColumnarExchange [c_customer_sk] #9 + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] + CometProject [ss_quantity,ss_sales_price,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] + CometProject [ss_customer_sk,ss_quantity,ss_sales_price] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometBroadcastExchange [c_customer_sk] #10 + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #15 + CometSortMergeJoin [c_customer_sk,c_first_name,c_last_name,c_customer_sk] + CometSort [c_customer_sk,c_first_name,c_last_name] + CometColumnarExchange [c_customer_sk] #16 + CometFilter [c_customer_sk,c_first_name,c_last_name] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] + CometSort [c_customer_sk] + CometProject [c_customer_sk] + CometFilter [c_customer_sk,ssales] + ReusedSubquery [tpcds_cmax] #3 + CometHashAggregate [c_customer_sk,ssales,sum,isEmpty,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + ReusedExchange [c_customer_sk,sum,isEmpty] #9 + CometBroadcastExchange [d_date_sk] #17 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometHashAggregate [c_last_name,c_first_name,sales,sum,isEmpty,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price))] + CometColumnarExchange [c_last_name,c_first_name] #18 + CometHashAggregate [c_last_name,c_first_name,sum,isEmpty,ws_quantity,ws_list_price] + CometProject [ws_quantity,ws_list_price,c_first_name,c_last_name] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name,d_date_sk] + CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk,c_first_name,c_last_name] + CometSortMergeJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk] + CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + CometColumnarExchange [ws_bill_customer_sk] #19 + CometProject [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,item_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [item_sk] #4 + CometSort [c_customer_sk] + CometProject [c_customer_sk] + CometFilter [c_customer_sk,ssales] + ReusedSubquery [tpcds_cmax] #3 + CometHashAggregate [c_customer_sk,ssales,sum,isEmpty,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + ReusedExchange [c_customer_sk,sum,isEmpty] #9 + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #15 + ReusedExchange [d_date_sk] #17 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt index 84411372c..549a3ae5a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt @@ -1,50 +1,50 @@ == Physical Plan == * Filter (46) +- * HashAggregate (45) - +- Exchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- Exchange (41) + +- * ColumnarToRow (44) + +- CometColumnarExchange (43) + +- RowToColumnar (42) + +- * HashAggregate (41) +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Project (14) - : : : : +- * SortMergeJoin Inner (13) - : : : : :- * Sort (6) - : : : : : +- Exchange (5) - : : : : : +- * ColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- * Sort (12) - : : : : +- Exchange (11) - : : : : +- * ColumnarToRow (10) - : : : : +- CometProject (9) - : : : : +- CometFilter (8) - : : : : +- CometScan parquet spark_catalog.default.store_returns (7) - : : : +- BroadcastExchange (19) - : : : +- * ColumnarToRow (18) - : : : +- CometProject (17) - : : : +- CometFilter (16) - : : : +- CometScan parquet spark_catalog.default.store (15) - : : +- BroadcastExchange (25) - : : +- * ColumnarToRow (24) - : : +- CometFilter (23) - : : +- CometScan parquet spark_catalog.default.item (22) - : +- BroadcastExchange (31) - : +- * ColumnarToRow (30) - : +- CometFilter (29) - : +- CometScan parquet spark_catalog.default.customer (28) - +- BroadcastExchange (37) - +- * ColumnarToRow (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.customer_address (34) + +- * ColumnarToRow (39) + +- CometColumnarExchange (38) + +- RowToColumnar (37) + +- * HashAggregate (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * ColumnarToRow (29) + : +- CometProject (28) + : +- CometBroadcastHashJoin (27) + : :- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometProject (18) + : : : +- CometBroadcastHashJoin (17) + : : : :- CometProject (12) + : : : : +- CometSortMergeJoin (11) + : : : : :- CometSort (5) + : : : : : +- CometColumnarExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometSort (10) + : : : : +- CometColumnarExchange (9) + : : : : +- CometProject (8) + : : : : +- CometFilter (7) + : : : : +- CometScan parquet spark_catalog.default.store_returns (6) + : : : +- CometBroadcastExchange (16) + : : : +- CometProject (15) + : : : +- CometFilter (14) + : : : +- CometScan parquet spark_catalog.default.store (13) + : : +- CometBroadcastExchange (21) + : : +- CometFilter (20) + : : +- CometScan parquet spark_catalog.default.item (19) + : +- CometBroadcastExchange (26) + : +- CometFilter (25) + : +- CometScan parquet spark_catalog.default.customer (24) + +- BroadcastExchange (33) + +- * ColumnarToRow (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.customer_address (30) (1) Scan parquet spark_catalog.default.store_sales @@ -62,277 +62,275 @@ Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND is Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -(4) ColumnarToRow [codegen id : 1] +(4) CometColumnarExchange Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(5) Exchange +(5) CometSort Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] -(6) Sort [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST], false, 0 - -(7) Scan parquet spark_catalog.default.store_returns +(6) Scan parquet spark_catalog.default.store_returns Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(8) CometFilter +(7) CometFilter Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) -(9) CometProject +(8) CometProject Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] -(10) ColumnarToRow [codegen id : 3] -Input [2]: [sr_item_sk#7, sr_ticket_number#8] - -(11) Exchange +(9) CometColumnarExchange Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(12) Sort [codegen id : 4] +(10) CometSort Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST], false, 0 +Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] -(13) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_ticket_number#4, ss_item_sk#1] -Right keys [2]: [sr_ticket_number#8, sr_item_sk#7] -Join type: Inner -Join condition: None +(11) CometSortMergeJoin +Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Right output [2]: [sr_item_sk#7, sr_ticket_number#8] +Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner -(14) Project [codegen id : 9] -Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +(12) CometProject Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -(15) Scan parquet spark_catalog.default.store +(13) Scan parquet spark_catalog.default.store Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(16) CometFilter +(14) CometFilter Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(s_zip#14)) -(17) CometProject +(15) CometProject Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Arguments: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14], [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(18) ColumnarToRow [codegen id : 5] +(16) CometBroadcastExchange Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(19) BroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(20) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#10] -Join type: Inner -Join condition: None +(17) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +Right output [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight -(21) Project [codegen id : 9] -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] +(18) CometProject Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] -(22) Scan parquet spark_catalog.default.item +(19) Scan parquet spark_catalog.default.item Output [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale ), IsNotNull(i_item_sk)] ReadSchema: struct -(23) CometFilter +(20) CometFilter Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Condition : ((isnotnull(i_color#18) AND (i_color#18 = pale )) AND isnotnull(i_item_sk#15)) -(24) ColumnarToRow [codegen id : 6] +(21) CometBroadcastExchange Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Arguments: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(25) BroadcastExchange -Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(22) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] +Right output [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Arguments: [ss_item_sk#1], [i_item_sk#15], Inner, BuildRight -(26) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#15] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 9] -Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +(23) CometProject Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(28) Scan parquet spark_catalog.default.customer +(24) Scan parquet spark_catalog.default.customer Output [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct -(29) CometFilter +(25) CometFilter Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_birth_country#24)) -(30) ColumnarToRow [codegen id : 7] -Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] - -(31) BroadcastExchange +(26) CometBroadcastExchange Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +Arguments: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] -(32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#21] -Join type: Inner -Join condition: None +(27) CometBroadcastHashJoin +Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Right output [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] +Arguments: [ss_customer_sk#2], [c_customer_sk#21], Inner, BuildRight -(33) Project [codegen id : 9] -Output [12]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] +(28) CometProject Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] +Arguments: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24], [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] -(34) Scan parquet spark_catalog.default.customer_address +(29) ColumnarToRow [codegen id : 2] +Input [12]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] + +(30) Scan parquet spark_catalog.default.customer_address Output [3]: [ca_state#25, ca_zip#26, ca_country#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct -(35) CometFilter +(31) CometFilter Input [3]: [ca_state#25, ca_zip#26, ca_country#27] Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) -(36) ColumnarToRow [codegen id : 8] +(32) ColumnarToRow [codegen id : 1] Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -(37) BroadcastExchange +(33) BroadcastExchange Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [plan_id=3] -(38) BroadcastHashJoin [codegen id : 9] +(34) BroadcastHashJoin [codegen id : 2] Left keys [2]: [c_birth_country#24, s_zip#14] Right keys [2]: [upper(ca_country#27), ca_zip#26] Join type: Inner Join condition: None -(39) Project [codegen id : 9] +(35) Project [codegen id : 2] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] Input [15]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24, ca_state#25, ca_zip#26, ca_country#27] -(40) HashAggregate [codegen id : 9] +(36) HashAggregate [codegen id : 2] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum#28] Results [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] -(41) Exchange +(37) RowToColumnar +Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] + +(38) CometColumnarExchange +Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] +Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(39) ColumnarToRow [codegen id : 3] Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] -Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(42) HashAggregate [codegen id : 10] +(40) HashAggregate [codegen id : 3] Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#30] Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#30,17,2) AS netpaid#31] -(43) HashAggregate [codegen id : 10] +(41) HashAggregate [codegen id : 3] Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, netpaid#31] Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11] Functions [1]: [partial_sum(netpaid#31)] Aggregate Attributes [2]: [sum#32, isEmpty#33] Results [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] -(44) Exchange +(42) RowToColumnar Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] -Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(45) HashAggregate [codegen id : 11] +(43) CometColumnarExchange +Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] +Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(44) ColumnarToRow [codegen id : 4] +Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] + +(45) HashAggregate [codegen id : 4] Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11] Functions [1]: [sum(netpaid#31)] Aggregate Attributes [1]: [sum(netpaid#31)#36] Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, sum(netpaid#31)#36 AS paid#37] -(46) Filter [codegen id : 11] +(46) Filter [codegen id : 4] Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, paid#37] Condition : (isnotnull(paid#37) AND (cast(paid#37 as decimal(33,8)) > cast(Subquery scalar-subquery#38, [id=#39] as decimal(33,8)))) ===== Subqueries ===== Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#38, [id=#39] -* HashAggregate (73) -+- Exchange (72) - +- * HashAggregate (71) - +- * HashAggregate (70) - +- Exchange (69) - +- * HashAggregate (68) - +- * Project (67) - +- * BroadcastHashJoin Inner BuildRight (66) - :- * Project (64) - : +- * BroadcastHashJoin Inner BuildRight (63) - : :- * Project (61) - : : +- * BroadcastHashJoin Inner BuildRight (60) - : : :- * Project (55) - : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : :- * Project (52) - : : : : +- * SortMergeJoin Inner (51) - : : : : :- * Sort (48) - : : : : : +- ReusedExchange (47) - : : : : +- * Sort (50) - : : : : +- ReusedExchange (49) - : : : +- ReusedExchange (53) - : : +- BroadcastExchange (59) - : : +- * ColumnarToRow (58) - : : +- CometFilter (57) - : : +- CometScan parquet spark_catalog.default.item (56) - : +- ReusedExchange (62) - +- ReusedExchange (65) - - -(47) ReusedExchange [Reuses operator id: 5] +* HashAggregate (77) ++- * ColumnarToRow (76) + +- CometColumnarExchange (75) + +- RowToColumnar (74) + +- * HashAggregate (73) + +- * HashAggregate (72) + +- * ColumnarToRow (71) + +- CometColumnarExchange (70) + +- RowToColumnar (69) + +- * HashAggregate (68) + +- * Project (67) + +- * BroadcastHashJoin Inner BuildRight (66) + :- * ColumnarToRow (64) + : +- CometProject (63) + : +- CometBroadcastHashJoin (62) + : :- CometProject (60) + : : +- CometBroadcastHashJoin (59) + : : :- CometProject (55) + : : : +- CometBroadcastHashJoin (54) + : : : :- CometProject (52) + : : : : +- CometSortMergeJoin (51) + : : : : :- CometSort (48) + : : : : : +- ReusedExchange (47) + : : : : +- CometSort (50) + : : : : +- ReusedExchange (49) + : : : +- ReusedExchange (53) + : : +- CometBroadcastExchange (58) + : : +- CometFilter (57) + : : +- CometScan parquet spark_catalog.default.item (56) + : +- ReusedExchange (61) + +- ReusedExchange (65) + + +(47) ReusedExchange [Reuses operator id: 4] Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -(48) Sort [codegen id : 2] +(48) CometSort Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST], false, 0 +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] -(49) ReusedExchange [Reuses operator id: 11] +(49) ReusedExchange [Reuses operator id: 9] Output [2]: [sr_item_sk#7, sr_ticket_number#8] -(50) Sort [codegen id : 4] +(50) CometSort Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST], false, 0 +Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] -(51) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_ticket_number#4, ss_item_sk#1] -Right keys [2]: [sr_ticket_number#8, sr_item_sk#7] -Join type: Inner -Join condition: None +(51) CometSortMergeJoin +Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Right output [2]: [sr_item_sk#7, sr_ticket_number#8] +Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner -(52) Project [codegen id : 9] -Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +(52) CometProject Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -(53) ReusedExchange [Reuses operator id: 19] +(53) ReusedExchange [Reuses operator id: 16] Output [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(54) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#10] -Join type: Inner -Join condition: None +(54) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +Right output [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight -(55) Project [codegen id : 9] -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] +(55) CometProject Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] (56) Scan parquet spark_catalog.default.item Output [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] @@ -345,79 +343,89 @@ ReadSchema: struct -(8) CometFilter +(7) CometFilter Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) -(9) CometProject +(8) CometProject Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] -(10) ColumnarToRow [codegen id : 3] -Input [2]: [sr_item_sk#7, sr_ticket_number#8] - -(11) Exchange +(9) CometColumnarExchange Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(12) Sort [codegen id : 4] +(10) CometSort Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST], false, 0 +Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] -(13) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_ticket_number#4, ss_item_sk#1] -Right keys [2]: [sr_ticket_number#8, sr_item_sk#7] -Join type: Inner -Join condition: None +(11) CometSortMergeJoin +Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Right output [2]: [sr_item_sk#7, sr_ticket_number#8] +Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner -(14) Project [codegen id : 9] -Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +(12) CometProject Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -(15) Scan parquet spark_catalog.default.store +(13) Scan parquet spark_catalog.default.store Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(16) CometFilter +(14) CometFilter Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(s_zip#14)) -(17) CometProject +(15) CometProject Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Arguments: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14], [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(18) ColumnarToRow [codegen id : 5] +(16) CometBroadcastExchange Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(19) BroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(20) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#10] -Join type: Inner -Join condition: None +(17) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +Right output [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight -(21) Project [codegen id : 9] -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] +(18) CometProject Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] -(22) Scan parquet spark_catalog.default.item +(19) Scan parquet spark_catalog.default.item Output [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_color), EqualTo(i_color,chiffon ), IsNotNull(i_item_sk)] ReadSchema: struct -(23) CometFilter +(20) CometFilter Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Condition : ((isnotnull(i_color#18) AND (i_color#18 = chiffon )) AND isnotnull(i_item_sk#15)) -(24) ColumnarToRow [codegen id : 6] +(21) CometBroadcastExchange Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Arguments: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(25) BroadcastExchange -Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(22) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] +Right output [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Arguments: [ss_item_sk#1], [i_item_sk#15], Inner, BuildRight -(26) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#15] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 9] -Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +(23) CometProject Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(28) Scan parquet spark_catalog.default.customer +(24) Scan parquet spark_catalog.default.customer Output [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct -(29) CometFilter +(25) CometFilter Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_birth_country#24)) -(30) ColumnarToRow [codegen id : 7] -Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] - -(31) BroadcastExchange +(26) CometBroadcastExchange Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +Arguments: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] -(32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#21] -Join type: Inner -Join condition: None +(27) CometBroadcastHashJoin +Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Right output [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] +Arguments: [ss_customer_sk#2], [c_customer_sk#21], Inner, BuildRight -(33) Project [codegen id : 9] -Output [12]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] +(28) CometProject Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] +Arguments: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24], [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] -(34) Scan parquet spark_catalog.default.customer_address +(29) ColumnarToRow [codegen id : 2] +Input [12]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] + +(30) Scan parquet spark_catalog.default.customer_address Output [3]: [ca_state#25, ca_zip#26, ca_country#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct -(35) CometFilter +(31) CometFilter Input [3]: [ca_state#25, ca_zip#26, ca_country#27] Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) -(36) ColumnarToRow [codegen id : 8] +(32) ColumnarToRow [codegen id : 1] Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -(37) BroadcastExchange +(33) BroadcastExchange Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [plan_id=3] -(38) BroadcastHashJoin [codegen id : 9] +(34) BroadcastHashJoin [codegen id : 2] Left keys [2]: [c_birth_country#24, s_zip#14] Right keys [2]: [upper(ca_country#27), ca_zip#26] Join type: Inner Join condition: None -(39) Project [codegen id : 9] +(35) Project [codegen id : 2] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] Input [15]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24, ca_state#25, ca_zip#26, ca_country#27] -(40) HashAggregate [codegen id : 9] +(36) HashAggregate [codegen id : 2] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum#28] Results [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] -(41) Exchange +(37) RowToColumnar +Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] + +(38) CometColumnarExchange +Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] +Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(39) ColumnarToRow [codegen id : 3] Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] -Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(42) HashAggregate [codegen id : 10] +(40) HashAggregate [codegen id : 3] Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#30] Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#30,17,2) AS netpaid#31] -(43) HashAggregate [codegen id : 10] +(41) HashAggregate [codegen id : 3] Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, netpaid#31] Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11] Functions [1]: [partial_sum(netpaid#31)] Aggregate Attributes [2]: [sum#32, isEmpty#33] Results [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] -(44) Exchange +(42) RowToColumnar Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] -Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(45) HashAggregate [codegen id : 11] +(43) CometColumnarExchange +Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] +Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(44) ColumnarToRow [codegen id : 4] +Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] + +(45) HashAggregate [codegen id : 4] Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11] Functions [1]: [sum(netpaid#31)] Aggregate Attributes [1]: [sum(netpaid#31)#36] Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, sum(netpaid#31)#36 AS paid#37] -(46) Filter [codegen id : 11] +(46) Filter [codegen id : 4] Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, paid#37] Condition : (isnotnull(paid#37) AND (cast(paid#37 as decimal(33,8)) > cast(Subquery scalar-subquery#38, [id=#39] as decimal(33,8)))) ===== Subqueries ===== Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#38, [id=#39] -* HashAggregate (73) -+- Exchange (72) - +- * HashAggregate (71) - +- * HashAggregate (70) - +- Exchange (69) - +- * HashAggregate (68) - +- * Project (67) - +- * BroadcastHashJoin Inner BuildRight (66) - :- * Project (64) - : +- * BroadcastHashJoin Inner BuildRight (63) - : :- * Project (61) - : : +- * BroadcastHashJoin Inner BuildRight (60) - : : :- * Project (55) - : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : :- * Project (52) - : : : : +- * SortMergeJoin Inner (51) - : : : : :- * Sort (48) - : : : : : +- ReusedExchange (47) - : : : : +- * Sort (50) - : : : : +- ReusedExchange (49) - : : : +- ReusedExchange (53) - : : +- BroadcastExchange (59) - : : +- * ColumnarToRow (58) - : : +- CometFilter (57) - : : +- CometScan parquet spark_catalog.default.item (56) - : +- ReusedExchange (62) - +- ReusedExchange (65) - - -(47) ReusedExchange [Reuses operator id: 5] +* HashAggregate (77) ++- * ColumnarToRow (76) + +- CometColumnarExchange (75) + +- RowToColumnar (74) + +- * HashAggregate (73) + +- * HashAggregate (72) + +- * ColumnarToRow (71) + +- CometColumnarExchange (70) + +- RowToColumnar (69) + +- * HashAggregate (68) + +- * Project (67) + +- * BroadcastHashJoin Inner BuildRight (66) + :- * ColumnarToRow (64) + : +- CometProject (63) + : +- CometBroadcastHashJoin (62) + : :- CometProject (60) + : : +- CometBroadcastHashJoin (59) + : : :- CometProject (55) + : : : +- CometBroadcastHashJoin (54) + : : : :- CometProject (52) + : : : : +- CometSortMergeJoin (51) + : : : : :- CometSort (48) + : : : : : +- ReusedExchange (47) + : : : : +- CometSort (50) + : : : : +- ReusedExchange (49) + : : : +- ReusedExchange (53) + : : +- CometBroadcastExchange (58) + : : +- CometFilter (57) + : : +- CometScan parquet spark_catalog.default.item (56) + : +- ReusedExchange (61) + +- ReusedExchange (65) + + +(47) ReusedExchange [Reuses operator id: 4] Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -(48) Sort [codegen id : 2] +(48) CometSort Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST], false, 0 +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] -(49) ReusedExchange [Reuses operator id: 11] +(49) ReusedExchange [Reuses operator id: 9] Output [2]: [sr_item_sk#7, sr_ticket_number#8] -(50) Sort [codegen id : 4] +(50) CometSort Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST], false, 0 +Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] -(51) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_ticket_number#4, ss_item_sk#1] -Right keys [2]: [sr_ticket_number#8, sr_item_sk#7] -Join type: Inner -Join condition: None +(51) CometSortMergeJoin +Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Right output [2]: [sr_item_sk#7, sr_ticket_number#8] +Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner -(52) Project [codegen id : 9] -Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +(52) CometProject Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -(53) ReusedExchange [Reuses operator id: 19] +(53) ReusedExchange [Reuses operator id: 16] Output [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(54) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#10] -Join type: Inner -Join condition: None +(54) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +Right output [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight -(55) Project [codegen id : 9] -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] +(55) CometProject Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] (56) Scan parquet spark_catalog.default.item Output [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] @@ -345,79 +343,89 @@ ReadSchema: struct -(12) CometFilter +(14) CometFilter Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] Condition : (((isnotnull(ss_quantity#16) AND (ss_quantity#16 >= 6)) AND (ss_quantity#16 <= 10)) AND ((((ss_list_price#18 >= 90.00) AND (ss_list_price#18 <= 100.00)) OR ((ss_coupon_amt#19 >= 2323.00) AND (ss_coupon_amt#19 <= 3323.00))) OR ((ss_wholesale_cost#17 >= 31.00) AND (ss_wholesale_cost#17 <= 51.00)))) -(13) CometProject +(15) CometProject Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] Arguments: [ss_list_price#18], [ss_list_price#18] -(14) CometHashAggregate +(16) CometHashAggregate Input [1]: [ss_list_price#18] Keys [1]: [ss_list_price#18] Functions [2]: [partial_avg(UnscaledValue(ss_list_price#18)), partial_count(ss_list_price#18)] -(15) ColumnarToRow [codegen id : 3] +(17) CometColumnarExchange Input [4]: [ss_list_price#18, sum#21, count#22, count#23] +Arguments: hashpartitioning(ss_list_price#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(16) Exchange +(18) ColumnarToRow [codegen id : 2] Input [4]: [ss_list_price#18, sum#21, count#22, count#23] -Arguments: hashpartitioning(ss_list_price#18, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(17) HashAggregate [codegen id : 4] +(19) HashAggregate [codegen id : 2] Input [4]: [ss_list_price#18, sum#21, count#22, count#23] Keys [1]: [ss_list_price#18] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#18))#24, count(ss_list_price#18)#25] Results [4]: [ss_list_price#18, sum#21, count#22, count#23] -(18) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 2] Input [4]: [ss_list_price#18, sum#21, count#22, count#23] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18), partial_count(distinct ss_list_price#18)] Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#18))#24, count(ss_list_price#18)#25, count(ss_list_price#18)#26] Results [4]: [sum#21, count#22, count#23, count#27] -(19) Exchange +(21) RowToColumnar Input [4]: [sum#21, count#22, count#23, count#27] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] -(20) HashAggregate [codegen id : 5] +(22) CometColumnarExchange +Input [4]: [sum#21, count#22, count#23, count#27] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(23) CometHashAggregate Input [4]: [sum#21, count#22, count#23, count#27] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#18)), count(ss_list_price#18), count(distinct ss_list_price#18)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#18))#24, count(ss_list_price#18)#25, count(ss_list_price#18)#26] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#18))#24 / 100.0) as decimal(11,6)) AS B2_LP#28, count(ss_list_price#18)#25 AS B2_CNT#29, count(ss_list_price#18)#26 AS B2_CNTD#30] -(21) BroadcastExchange +(24) ColumnarToRow [codegen id : 3] +Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] + +(25) BroadcastExchange Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] Arguments: IdentityBroadcastMode, [plan_id=5] -(22) BroadcastNestedLoopJoin [codegen id : 18] +(26) BroadcastNestedLoopJoin [codegen id : 12] Join type: Inner Join condition: None -(23) Scan parquet spark_catalog.default.store_sales +(27) Scan parquet spark_catalog.default.store_sales Output [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,11), LessThanOrEqual(ss_quantity,15), Or(Or(And(GreaterThanOrEqual(ss_list_price,142.00),LessThanOrEqual(ss_list_price,152.00)),And(GreaterThanOrEqual(ss_coupon_amt,12214.00),LessThanOrEqual(ss_coupon_amt,13214.00))),And(GreaterThanOrEqual(ss_wholesale_cost,79.00),LessThanOrEqual(ss_wholesale_cost,99.00)))] ReadSchema: struct -(24) CometFilter +(28) CometFilter Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] Condition : (((isnotnull(ss_quantity#31) AND (ss_quantity#31 >= 11)) AND (ss_quantity#31 <= 15)) AND ((((ss_list_price#33 >= 142.00) AND (ss_list_price#33 <= 152.00)) OR ((ss_coupon_amt#34 >= 12214.00) AND (ss_coupon_amt#34 <= 13214.00))) OR ((ss_wholesale_cost#32 >= 79.00) AND (ss_wholesale_cost#32 <= 99.00)))) -(25) CometProject +(29) CometProject Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] Arguments: [ss_list_price#33], [ss_list_price#33] -(26) CometHashAggregate +(30) CometHashAggregate Input [1]: [ss_list_price#33] Keys [1]: [ss_list_price#33] Functions [2]: [partial_avg(UnscaledValue(ss_list_price#33)), partial_count(ss_list_price#33)] -(27) ColumnarToRow [codegen id : 6] +(31) CometColumnarExchange Input [4]: [ss_list_price#33, sum#36, count#37, count#38] +Arguments: hashpartitioning(ss_list_price#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(28) Exchange +(32) ColumnarToRow [codegen id : 4] Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -Arguments: hashpartitioning(ss_list_price#33, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(29) HashAggregate [codegen id : 7] +(33) HashAggregate [codegen id : 4] Input [4]: [ss_list_price#33, sum#36, count#37, count#38] Keys [1]: [ss_list_price#33] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40] Results [4]: [ss_list_price#33, sum#36, count#37, count#38] -(30) HashAggregate [codegen id : 7] +(34) HashAggregate [codegen id : 4] Input [4]: [ss_list_price#33, sum#36, count#37, count#38] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33), partial_count(distinct ss_list_price#33)] Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40, count(ss_list_price#33)#41] Results [4]: [sum#36, count#37, count#38, count#42] -(31) Exchange +(35) RowToColumnar +Input [4]: [sum#36, count#37, count#38, count#42] + +(36) CometColumnarExchange Input [4]: [sum#36, count#37, count#38, count#42] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(32) HashAggregate [codegen id : 8] +(37) CometHashAggregate Input [4]: [sum#36, count#37, count#38, count#42] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#33)), count(ss_list_price#33), count(distinct ss_list_price#33)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40, count(ss_list_price#33)#41] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#33))#39 / 100.0) as decimal(11,6)) AS B3_LP#43, count(ss_list_price#33)#40 AS B3_CNT#44, count(ss_list_price#33)#41 AS B3_CNTD#45] -(33) BroadcastExchange +(38) ColumnarToRow [codegen id : 5] +Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] + +(39) BroadcastExchange Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] Arguments: IdentityBroadcastMode, [plan_id=8] -(34) BroadcastNestedLoopJoin [codegen id : 18] +(40) BroadcastNestedLoopJoin [codegen id : 12] Join type: Inner Join condition: None -(35) Scan parquet spark_catalog.default.store_sales +(41) Scan parquet spark_catalog.default.store_sales Output [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,16), LessThanOrEqual(ss_quantity,20), Or(Or(And(GreaterThanOrEqual(ss_list_price,135.00),LessThanOrEqual(ss_list_price,145.00)),And(GreaterThanOrEqual(ss_coupon_amt,6071.00),LessThanOrEqual(ss_coupon_amt,7071.00))),And(GreaterThanOrEqual(ss_wholesale_cost,38.00),LessThanOrEqual(ss_wholesale_cost,58.00)))] ReadSchema: struct -(36) CometFilter +(42) CometFilter Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] Condition : (((isnotnull(ss_quantity#46) AND (ss_quantity#46 >= 16)) AND (ss_quantity#46 <= 20)) AND ((((ss_list_price#48 >= 135.00) AND (ss_list_price#48 <= 145.00)) OR ((ss_coupon_amt#49 >= 6071.00) AND (ss_coupon_amt#49 <= 7071.00))) OR ((ss_wholesale_cost#47 >= 38.00) AND (ss_wholesale_cost#47 <= 58.00)))) -(37) CometProject +(43) CometProject Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] Arguments: [ss_list_price#48], [ss_list_price#48] -(38) CometHashAggregate +(44) CometHashAggregate Input [1]: [ss_list_price#48] Keys [1]: [ss_list_price#48] Functions [2]: [partial_avg(UnscaledValue(ss_list_price#48)), partial_count(ss_list_price#48)] -(39) ColumnarToRow [codegen id : 9] +(45) CometColumnarExchange Input [4]: [ss_list_price#48, sum#51, count#52, count#53] +Arguments: hashpartitioning(ss_list_price#48, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(40) Exchange +(46) ColumnarToRow [codegen id : 6] Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -Arguments: hashpartitioning(ss_list_price#48, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(41) HashAggregate [codegen id : 10] +(47) HashAggregate [codegen id : 6] Input [4]: [ss_list_price#48, sum#51, count#52, count#53] Keys [1]: [ss_list_price#48] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55] Results [4]: [ss_list_price#48, sum#51, count#52, count#53] -(42) HashAggregate [codegen id : 10] +(48) HashAggregate [codegen id : 6] Input [4]: [ss_list_price#48, sum#51, count#52, count#53] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48), partial_count(distinct ss_list_price#48)] Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55, count(ss_list_price#48)#56] Results [4]: [sum#51, count#52, count#53, count#57] -(43) Exchange +(49) RowToColumnar Input [4]: [sum#51, count#52, count#53, count#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] -(44) HashAggregate [codegen id : 11] +(50) CometColumnarExchange +Input [4]: [sum#51, count#52, count#53, count#57] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(51) CometHashAggregate Input [4]: [sum#51, count#52, count#53, count#57] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#48)), count(ss_list_price#48), count(distinct ss_list_price#48)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55, count(ss_list_price#48)#56] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#48))#54 / 100.0) as decimal(11,6)) AS B4_LP#58, count(ss_list_price#48)#55 AS B4_CNT#59, count(ss_list_price#48)#56 AS B4_CNTD#60] -(45) BroadcastExchange +(52) ColumnarToRow [codegen id : 7] +Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] + +(53) BroadcastExchange Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] Arguments: IdentityBroadcastMode, [plan_id=11] -(46) BroadcastNestedLoopJoin [codegen id : 18] +(54) BroadcastNestedLoopJoin [codegen id : 12] Join type: Inner Join condition: None -(47) Scan parquet spark_catalog.default.store_sales +(55) Scan parquet spark_catalog.default.store_sales Output [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,25), Or(Or(And(GreaterThanOrEqual(ss_list_price,122.00),LessThanOrEqual(ss_list_price,132.00)),And(GreaterThanOrEqual(ss_coupon_amt,836.00),LessThanOrEqual(ss_coupon_amt,1836.00))),And(GreaterThanOrEqual(ss_wholesale_cost,17.00),LessThanOrEqual(ss_wholesale_cost,37.00)))] ReadSchema: struct -(48) CometFilter +(56) CometFilter Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] Condition : (((isnotnull(ss_quantity#61) AND (ss_quantity#61 >= 21)) AND (ss_quantity#61 <= 25)) AND ((((ss_list_price#63 >= 122.00) AND (ss_list_price#63 <= 132.00)) OR ((ss_coupon_amt#64 >= 836.00) AND (ss_coupon_amt#64 <= 1836.00))) OR ((ss_wholesale_cost#62 >= 17.00) AND (ss_wholesale_cost#62 <= 37.00)))) -(49) CometProject +(57) CometProject Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] Arguments: [ss_list_price#63], [ss_list_price#63] -(50) CometHashAggregate +(58) CometHashAggregate Input [1]: [ss_list_price#63] Keys [1]: [ss_list_price#63] Functions [2]: [partial_avg(UnscaledValue(ss_list_price#63)), partial_count(ss_list_price#63)] -(51) ColumnarToRow [codegen id : 12] +(59) CometColumnarExchange Input [4]: [ss_list_price#63, sum#66, count#67, count#68] +Arguments: hashpartitioning(ss_list_price#63, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(52) Exchange +(60) ColumnarToRow [codegen id : 8] Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -Arguments: hashpartitioning(ss_list_price#63, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(53) HashAggregate [codegen id : 13] +(61) HashAggregate [codegen id : 8] Input [4]: [ss_list_price#63, sum#66, count#67, count#68] Keys [1]: [ss_list_price#63] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70] Results [4]: [ss_list_price#63, sum#66, count#67, count#68] -(54) HashAggregate [codegen id : 13] +(62) HashAggregate [codegen id : 8] Input [4]: [ss_list_price#63, sum#66, count#67, count#68] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63), partial_count(distinct ss_list_price#63)] Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70, count(ss_list_price#63)#71] Results [4]: [sum#66, count#67, count#68, count#72] -(55) Exchange +(63) RowToColumnar Input [4]: [sum#66, count#67, count#68, count#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] -(56) HashAggregate [codegen id : 14] +(64) CometColumnarExchange +Input [4]: [sum#66, count#67, count#68, count#72] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] + +(65) CometHashAggregate Input [4]: [sum#66, count#67, count#68, count#72] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#63)), count(ss_list_price#63), count(distinct ss_list_price#63)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70, count(ss_list_price#63)#71] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#63))#69 / 100.0) as decimal(11,6)) AS B5_LP#73, count(ss_list_price#63)#70 AS B5_CNT#74, count(ss_list_price#63)#71 AS B5_CNTD#75] -(57) BroadcastExchange +(66) ColumnarToRow [codegen id : 9] +Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] + +(67) BroadcastExchange Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] Arguments: IdentityBroadcastMode, [plan_id=14] -(58) BroadcastNestedLoopJoin [codegen id : 18] +(68) BroadcastNestedLoopJoin [codegen id : 12] Join type: Inner Join condition: None -(59) Scan parquet spark_catalog.default.store_sales +(69) Scan parquet spark_catalog.default.store_sales Output [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,26), LessThanOrEqual(ss_quantity,30), Or(Or(And(GreaterThanOrEqual(ss_list_price,154.00),LessThanOrEqual(ss_list_price,164.00)),And(GreaterThanOrEqual(ss_coupon_amt,7326.00),LessThanOrEqual(ss_coupon_amt,8326.00))),And(GreaterThanOrEqual(ss_wholesale_cost,7.00),LessThanOrEqual(ss_wholesale_cost,27.00)))] ReadSchema: struct -(60) CometFilter +(70) CometFilter Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] Condition : (((isnotnull(ss_quantity#76) AND (ss_quantity#76 >= 26)) AND (ss_quantity#76 <= 30)) AND ((((ss_list_price#78 >= 154.00) AND (ss_list_price#78 <= 164.00)) OR ((ss_coupon_amt#79 >= 7326.00) AND (ss_coupon_amt#79 <= 8326.00))) OR ((ss_wholesale_cost#77 >= 7.00) AND (ss_wholesale_cost#77 <= 27.00)))) -(61) CometProject +(71) CometProject Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] Arguments: [ss_list_price#78], [ss_list_price#78] -(62) CometHashAggregate +(72) CometHashAggregate Input [1]: [ss_list_price#78] Keys [1]: [ss_list_price#78] Functions [2]: [partial_avg(UnscaledValue(ss_list_price#78)), partial_count(ss_list_price#78)] -(63) ColumnarToRow [codegen id : 15] +(73) CometColumnarExchange Input [4]: [ss_list_price#78, sum#81, count#82, count#83] +Arguments: hashpartitioning(ss_list_price#78, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] -(64) Exchange +(74) ColumnarToRow [codegen id : 10] Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -Arguments: hashpartitioning(ss_list_price#78, 5), ENSURE_REQUIREMENTS, [plan_id=15] -(65) HashAggregate [codegen id : 16] +(75) HashAggregate [codegen id : 10] Input [4]: [ss_list_price#78, sum#81, count#82, count#83] Keys [1]: [ss_list_price#78] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85] Results [4]: [ss_list_price#78, sum#81, count#82, count#83] -(66) HashAggregate [codegen id : 16] +(76) HashAggregate [codegen id : 10] Input [4]: [ss_list_price#78, sum#81, count#82, count#83] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78), partial_count(distinct ss_list_price#78)] Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85, count(ss_list_price#78)#86] Results [4]: [sum#81, count#82, count#83, count#87] -(67) Exchange +(77) RowToColumnar +Input [4]: [sum#81, count#82, count#83, count#87] + +(78) CometColumnarExchange Input [4]: [sum#81, count#82, count#83, count#87] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=16] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] -(68) HashAggregate [codegen id : 17] +(79) CometHashAggregate Input [4]: [sum#81, count#82, count#83, count#87] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#78)), count(ss_list_price#78), count(distinct ss_list_price#78)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85, count(ss_list_price#78)#86] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#78))#84 / 100.0) as decimal(11,6)) AS B6_LP#88, count(ss_list_price#78)#85 AS B6_CNT#89, count(ss_list_price#78)#86 AS B6_CNTD#90] -(69) BroadcastExchange +(80) ColumnarToRow [codegen id : 11] +Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] + +(81) BroadcastExchange Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] Arguments: IdentityBroadcastMode, [plan_id=17] -(70) BroadcastNestedLoopJoin [codegen id : 18] +(82) BroadcastNestedLoopJoin [codegen id : 12] Join type: Inner Join condition: None diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt index 80c886708..73343f39e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt @@ -1,110 +1,110 @@ -WholeStageCodegen (18) +WholeStageCodegen (12) BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B1_LP,B1_CNT,B1_CNTD,sum,count,count,count] + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (2) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + CometHashAggregate [B1_LP,B1_CNT,B1_CNTD,sum,count,count,count,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] + CometColumnarExchange #1 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + ColumnarToRow + InputAdapter + CometColumnarExchange [ss_list_price] #2 CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #3 - WholeStageCodegen (5) - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B2_LP,B2_CNT,B2_CNTD,sum,count,count,count] + WholeStageCodegen (3) + ColumnarToRow InputAdapter - Exchange #4 - WholeStageCodegen (4) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter + CometHashAggregate [B2_LP,B2_CNT,B2_CNTD,sum,count,count,count,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] + CometColumnarExchange #4 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + ColumnarToRow + InputAdapter + CometColumnarExchange [ss_list_price] #5 CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #6 - WholeStageCodegen (8) - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B3_LP,B3_CNT,B3_CNTD,sum,count,count,count] + WholeStageCodegen (5) + ColumnarToRow InputAdapter - Exchange #7 - WholeStageCodegen (7) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #8 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter + CometHashAggregate [B3_LP,B3_CNT,B3_CNTD,sum,count,count,count,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] + CometColumnarExchange #7 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + ColumnarToRow + InputAdapter + CometColumnarExchange [ss_list_price] #8 CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #9 - WholeStageCodegen (11) - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B4_LP,B4_CNT,B4_CNTD,sum,count,count,count] + WholeStageCodegen (7) + ColumnarToRow InputAdapter - Exchange #10 - WholeStageCodegen (10) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #11 - WholeStageCodegen (9) - ColumnarToRow - InputAdapter + CometHashAggregate [B4_LP,B4_CNT,B4_CNTD,sum,count,count,count,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] + CometColumnarExchange #10 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + ColumnarToRow + InputAdapter + CometColumnarExchange [ss_list_price] #11 CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #12 - WholeStageCodegen (14) - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B5_LP,B5_CNT,B5_CNTD,sum,count,count,count] + WholeStageCodegen (9) + ColumnarToRow InputAdapter - Exchange #13 - WholeStageCodegen (13) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #14 - WholeStageCodegen (12) - ColumnarToRow - InputAdapter + CometHashAggregate [B5_LP,B5_CNT,B5_CNTD,sum,count,count,count,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] + CometColumnarExchange #13 + RowToColumnar + WholeStageCodegen (8) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + ColumnarToRow + InputAdapter + CometColumnarExchange [ss_list_price] #14 CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #15 - WholeStageCodegen (17) - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B6_LP,B6_CNT,B6_CNTD,sum,count,count,count] + WholeStageCodegen (11) + ColumnarToRow InputAdapter - Exchange #16 - WholeStageCodegen (16) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - InputAdapter - Exchange [ss_list_price] #17 - WholeStageCodegen (15) - ColumnarToRow - InputAdapter + CometHashAggregate [B6_LP,B6_CNT,B6_CNTD,sum,count,count,count,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] + CometColumnarExchange #16 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + ColumnarToRow + InputAdapter + CometColumnarExchange [ss_list_price] #17 CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] 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 3c350ad07..a3346549e 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,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (45) -+- * HashAggregate (44) - +- Exchange (43) - +- * ColumnarToRow (42) +* ColumnarToRow (45) ++- CometTakeOrderedAndProject (44) + +- CometHashAggregate (43) + +- CometColumnarExchange (42) +- CometHashAggregate (41) +- CometProject (40) +- CometBroadcastHashJoin (39) @@ -245,23 +245,21 @@ Input [7]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#28, 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)] -(42) ColumnarToRow [codegen id : 1] +(42) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(43) CometHashAggregate 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)#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] -(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] +(44) CometTakeOrderedAndProject +Input [7]: [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29, store_sales_quantity#36, store_returns_quantity#37, catalog_sales_quantity#38] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[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], output=[i_item_id#31,i_item_desc#32,s_store_id#28,s_store_name#29,store_sales_quantity#36,store_returns_quantity#37,catalog_sales_quantity#38]), [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29, store_sales_quantity#36, store_returns_quantity#37, catalog_sales_quantity#38], 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#36, store_returns_quantity#37, catalog_sales_quantity#38] + +(45) ColumnarToRow [codegen id : 1] +Input [7]: [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29, store_sales_quantity#36, store_returns_quantity#37, catalog_sales_quantity#38] ===== Subqueries ===== 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 88b4ecd1d..3ca13e7b4 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,73 +1,71 @@ -TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum,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,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc] - CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_store_id,s_store_name] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,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 [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_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_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_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 [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,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 - CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - 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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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 + CometTakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity] + CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity,sum,sum,sum,sum(ss_quantity),sum(sr_return_quantity),sum(cs_quantity)] + CometColumnarExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 + CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum,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,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc] + CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_store_id,s_store_name] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,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 [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_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_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_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 [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,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + 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_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #8 + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #9 + CometBroadcastExchange [d_date_sk] #8 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #10 - CometFilter [s_store_sk,s_store_id,s_store_name] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #11 - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk] #9 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #10 + CometFilter [s_store_sk,s_store_id,s_store_name] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #11 + CometFilter [i_item_sk,i_item_id,i_item_desc] + 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 98092ef29..9079e0886 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,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (19) -+- * HashAggregate (18) - +- Exchange (17) - +- * ColumnarToRow (16) +* ColumnarToRow (19) ++- CometTakeOrderedAndProject (18) + +- CometHashAggregate (17) + +- CometColumnarExchange (16) +- CometHashAggregate (15) +- CometProject (14) +- CometBroadcastHashJoin (13) @@ -93,21 +93,19 @@ 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))] -(16) ColumnarToRow [codegen id : 1] +(16) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(18) HashAggregate [codegen id : 2] +(17) CometHashAggregate 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))#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] -(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] +(18) CometTakeOrderedAndProject +Input [4]: [d_year#2, brand_id#12, brand#13, sum_agg#14] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,sum_agg#14 DESC NULLS LAST,brand_id#12 ASC NULLS FIRST], output=[d_year#2,brand_id#12,brand#13,sum_agg#14]), [d_year#2, brand_id#12, brand#13, sum_agg#14], 100, [d_year#2 ASC NULLS FIRST, sum_agg#14 DESC NULLS LAST, brand_id#12 ASC NULLS FIRST], [d_year#2, brand_id#12, brand#13, sum_agg#14] + +(19) ColumnarToRow [codegen id : 1] +Input [4]: [d_year#2, brand_id#12, brand#13, sum_agg#14] 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 bccda58a4..65c80baed 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,23 +1,21 @@ -TakeOrderedAndProject [d_year,sum_agg,brand_id,brand] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [d_year,i_brand,i_brand_id,sum,ss_ext_sales_price] - CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] - CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [d_year,brand_id,brand,sum_agg] + CometHashAggregate [d_year,brand_id,brand,sum_agg,i_brand,i_brand_id,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange [d_year,i_brand,i_brand_id] #1 + CometHashAggregate [d_year,i_brand,i_brand_id,sum,ss_ext_sales_price] + CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] + CometProject [d_year,ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometProject [d_date_sk,d_year] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 + CometProject [i_item_sk,i_brand_id,i_brand] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id] + 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 55c49d828..f16b49407 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,55 +1,52 @@ == Physical Plan == -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) +* ColumnarToRow (48) ++- CometTakeOrderedAndProject (47) + +- CometProject (46) + +- CometBroadcastHashJoin (45) + :- CometProject (40) + : +- CometBroadcastHashJoin (39) + : :- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometFilter (17) + : : : +- CometHashAggregate (16) + : : : +- CometColumnarExchange (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) + : : +- CometBroadcastExchange (33) + : : +- CometFilter (32) + : : +- CometHashAggregate (31) + : : +- CometColumnarExchange (30) + : : +- CometHashAggregate (29) + : : +- CometHashAggregate (28) + : : +- CometColumnarExchange (27) + : : +- CometHashAggregate (26) + : : +- CometProject (25) + : : +- CometBroadcastHashJoin (24) + : : :- CometProject (22) + : : : +- CometBroadcastHashJoin (21) + : : : :- CometFilter (19) + : : : : +- CometScan parquet spark_catalog.default.web_returns (18) + : : : +- ReusedExchange (20) + : : +- ReusedExchange (23) + : +- CometBroadcastExchange (38) + : +- CometFilter (37) + : +- CometScan parquet spark_catalog.default.customer (36) + +- CometBroadcastExchange (44) + +- CometProject (43) + +- CometFilter (42) + +- CometScan parquet spark_catalog.default.customer_address (41) (1) Scan parquet spark_catalog.default.web_returns @@ -121,211 +118,191 @@ 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))] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(17) HashAggregate [codegen id : 7] +(16) CometHashAggregate 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#9 AS ctr_state#13, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#11,17,2) AS ctr_total_return#14] -(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) CometFilter +Input [3]: [ctr_customer_sk#11, ctr_state#12, ctr_total_return#13] +Condition : isnotnull(ctr_total_return#13) -(19) Scan parquet spark_catalog.default.web_returns +(18) 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 [] -PartitionFilters: [isnotnull(wr_returned_date_sk#4), dynamicpruningexpression(wr_returned_date_sk#4 IN dynamicpruning#15)] +PartitionFilters: [isnotnull(wr_returned_date_sk#4), dynamicpruningexpression(wr_returned_date_sk#4 IN dynamicpruning#14)] PushedFilters: [IsNotNull(wr_returning_addr_sk)] ReadSchema: struct -(20) CometFilter +(19) 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) -(21) ReusedExchange [Reuses operator id: 6] +(20) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#6] -(22) CometBroadcastHashJoin +(21) 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 -(23) CometProject +(22) 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] -(24) ReusedExchange [Reuses operator id: 11] +(23) ReusedExchange [Reuses operator id: 11] Output [2]: [ca_address_sk#8, ca_state#9] -(25) CometBroadcastHashJoin +(24) 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 -(26) CometProject +(25) 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] -(27) CometHashAggregate +(26) 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))] -(28) ColumnarToRow [codegen id : 2] -Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#16] +(27) CometColumnarExchange +Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#15] +Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(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] +(28) CometHashAggregate +Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#15] 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#9 AS ctr_state#13, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#11,17,2) AS ctr_total_return#14] - -(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#17, count#18] -Results [3]: [ctr_state#13, sum#19, count#20] - -(32) Exchange -Input [3]: [ctr_state#13, sum#19, count#20] -Arguments: hashpartitioning(ctr_state#13, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(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)#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] - -(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) - -(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] - -(36) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ctr_state#13] -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)#22) - -(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)#22, ctr_state#13#23] - -(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] + +(29) CometHashAggregate +Input [2]: [ctr_state#12, ctr_total_return#13] +Keys [1]: [ctr_state#12] +Functions [1]: [partial_avg(ctr_total_return#13)] + +(30) CometColumnarExchange +Input [3]: [ctr_state#12, sum#16, count#17] +Arguments: hashpartitioning(ctr_state#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(31) CometHashAggregate +Input [3]: [ctr_state#12, sum#16, count#17] +Keys [1]: [ctr_state#12] +Functions [1]: [avg(ctr_total_return#13)] + +(32) CometFilter +Input [2]: [(avg(ctr_total_return) * 1.2)#18, ctr_state#12#19] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#18) + +(33) CometBroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#18, ctr_state#12#19] +Arguments: [(avg(ctr_total_return) * 1.2)#18, ctr_state#12#19] + +(34) CometBroadcastHashJoin +Left output [3]: [ctr_customer_sk#11, ctr_state#12, ctr_total_return#13] +Right output [2]: [(avg(ctr_total_return) * 1.2)#18, ctr_state#12#19] +Arguments: [ctr_state#12], [ctr_state#12#19], Inner, (cast(ctr_total_return#13 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#18), BuildRight + +(35) CometProject +Input [5]: [ctr_customer_sk#11, ctr_state#12, ctr_total_return#13, (avg(ctr_total_return) * 1.2)#18, ctr_state#12#19] +Arguments: [ctr_customer_sk#11, ctr_total_return#13], [ctr_customer_sk#11, ctr_total_return#13] + +(36) Scan parquet spark_catalog.default.customer +Output [14]: [c_customer_sk#20, c_customer_id#21, c_current_addr_sk#22, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_day#27, c_birth_month#28, c_birth_year#29, c_birth_country#30, c_login#31, c_email_address#32, c_last_review_date#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(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)) +(37) CometFilter +Input [14]: [c_customer_sk#20, c_customer_id#21, c_current_addr_sk#22, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_day#27, c_birth_month#28, c_birth_year#29, c_birth_country#30, c_login#31, c_email_address#32, c_last_review_date#33] +Condition : (isnotnull(c_customer_sk#20) AND isnotnull(c_current_addr_sk#22)) -(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] +(38) CometBroadcastExchange +Input [14]: [c_customer_sk#20, c_customer_id#21, c_current_addr_sk#22, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_day#27, c_birth_month#28, c_birth_year#29, c_birth_country#30, c_login#31, c_email_address#32, c_last_review_date#33] +Arguments: [c_customer_sk#20, c_customer_id#21, c_current_addr_sk#22, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_day#27, c_birth_month#28, c_birth_year#29, c_birth_country#30, c_login#31, c_email_address#32, c_last_review_date#33] -(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] +(39) CometBroadcastHashJoin +Left output [2]: [ctr_customer_sk#11, ctr_total_return#13] +Right output [14]: [c_customer_sk#20, c_customer_id#21, c_current_addr_sk#22, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_day#27, c_birth_month#28, c_birth_year#29, c_birth_country#30, c_login#31, c_email_address#32, c_last_review_date#33] +Arguments: [ctr_customer_sk#11], [c_customer_sk#20], Inner, BuildRight -(42) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ctr_customer_sk#12] -Right keys [1]: [c_customer_sk#24] -Join type: Inner -Join condition: None +(40) CometProject +Input [16]: [ctr_customer_sk#11, ctr_total_return#13, c_customer_sk#20, c_customer_id#21, c_current_addr_sk#22, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_day#27, c_birth_month#28, c_birth_year#29, c_birth_country#30, c_login#31, c_email_address#32, c_last_review_date#33] +Arguments: [ctr_total_return#13, c_customer_id#21, c_current_addr_sk#22, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_day#27, c_birth_month#28, c_birth_year#29, c_birth_country#30, c_login#31, c_email_address#32, c_last_review_date#33], [ctr_total_return#13, c_customer_id#21, c_current_addr_sk#22, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_day#27, c_birth_month#28, c_birth_year#29, c_birth_country#30, c_login#31, c_email_address#32, c_last_review_date#33] -(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] - -(44) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#38, ca_state#39] +(41) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#34, ca_state#35] 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 -(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)) +(42) CometFilter +Input [2]: [ca_address_sk#34, ca_state#35] +Condition : ((isnotnull(ca_state#35) AND (ca_state#35 = GA)) AND isnotnull(ca_address_sk#34)) -(46) CometProject -Input [2]: [ca_address_sk#38, ca_state#39] -Arguments: [ca_address_sk#38], [ca_address_sk#38] +(43) CometProject +Input [2]: [ca_address_sk#34, ca_state#35] +Arguments: [ca_address_sk#34], [ca_address_sk#34] -(47) ColumnarToRow [codegen id : 6] -Input [1]: [ca_address_sk#38] +(44) CometBroadcastExchange +Input [1]: [ca_address_sk#34] +Arguments: [ca_address_sk#34] -(48) BroadcastExchange -Input [1]: [ca_address_sk#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +(45) CometBroadcastHashJoin +Left output [14]: [ctr_total_return#13, c_customer_id#21, c_current_addr_sk#22, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_day#27, c_birth_month#28, c_birth_year#29, c_birth_country#30, c_login#31, c_email_address#32, c_last_review_date#33] +Right output [1]: [ca_address_sk#34] +Arguments: [c_current_addr_sk#22], [ca_address_sk#34], Inner, BuildRight -(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 +(46) CometProject +Input [15]: [ctr_total_return#13, c_customer_id#21, c_current_addr_sk#22, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_day#27, c_birth_month#28, c_birth_year#29, c_birth_country#30, c_login#31, c_email_address#32, c_last_review_date#33, ca_address_sk#34] +Arguments: [c_customer_id#21, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_day#27, c_birth_month#28, c_birth_year#29, c_birth_country#30, c_login#31, c_email_address#32, c_last_review_date#33, ctr_total_return#13], [c_customer_id#21, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_day#27, c_birth_month#28, c_birth_year#29, c_birth_country#30, c_login#31, c_email_address#32, c_last_review_date#33, ctr_total_return#13] -(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] +(47) CometTakeOrderedAndProject +Input [13]: [c_customer_id#21, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_day#27, c_birth_month#28, c_birth_year#29, c_birth_country#30, c_login#31, c_email_address#32, c_last_review_date#33, ctr_total_return#13] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#21 ASC NULLS FIRST,c_salutation#23 ASC NULLS FIRST,c_first_name#24 ASC NULLS FIRST,c_last_name#25 ASC NULLS FIRST,c_preferred_cust_flag#26 ASC NULLS FIRST,c_birth_day#27 ASC NULLS FIRST,c_birth_month#28 ASC NULLS FIRST,c_birth_year#29 ASC NULLS FIRST,c_birth_country#30 ASC NULLS FIRST,c_login#31 ASC NULLS FIRST,c_email_address#32 ASC NULLS FIRST,c_last_review_date#33 ASC NULLS FIRST,ctr_total_return#13 ASC NULLS FIRST], output=[c_customer_id#21,c_salutation#23,c_first_name#24,c_last_name#25,c_preferred_cust_flag#26,c_birth_day#27,c_birth_month#28,c_birth_year#29,c_birth_country#30,c_login#31,c_email_address#32,c_last_review_date#33,ctr_total_return#13]), [c_customer_id#21, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_day#27, c_birth_month#28, c_birth_year#29, c_birth_country#30, c_login#31, c_email_address#32, c_last_review_date#33, ctr_total_return#13], 100, [c_customer_id#21 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_last_name#25 ASC NULLS FIRST, c_preferred_cust_flag#26 ASC NULLS FIRST, c_birth_day#27 ASC NULLS FIRST, c_birth_month#28 ASC NULLS FIRST, c_birth_year#29 ASC NULLS FIRST, c_birth_country#30 ASC NULLS FIRST, c_login#31 ASC NULLS FIRST, c_email_address#32 ASC NULLS FIRST, c_last_review_date#33 ASC NULLS FIRST, ctr_total_return#13 ASC NULLS FIRST], [c_customer_id#21, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_day#27, c_birth_month#28, c_birth_year#29, c_birth_country#30, c_login#31, c_email_address#32, c_last_review_date#33, ctr_total_return#13] -(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] +(48) ColumnarToRow [codegen id : 1] +Input [13]: [c_customer_id#21, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_day#27, c_birth_month#28, c_birth_year#29, c_birth_country#30, c_login#31, c_email_address#32, c_last_review_date#33, ctr_total_return#13] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (56) -+- * ColumnarToRow (55) - +- CometProject (54) - +- CometFilter (53) - +- CometScan parquet spark_catalog.default.date_dim (52) +BroadcastExchange (53) ++- * ColumnarToRow (52) + +- CometProject (51) + +- CometFilter (50) + +- CometScan parquet spark_catalog.default.date_dim (49) -(52) Scan parquet spark_catalog.default.date_dim +(49) 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 -(53) CometFilter +(50) 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)) -(54) CometProject +(51) CometProject Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(55) ColumnarToRow [codegen id : 1] +(52) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(56) BroadcastExchange +(53) BroadcastExchange Input [1]: [d_date_sk#6] -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=4] -Subquery:2 Hosting operator id = 19 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 18 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 60cad82dd..f1bb3a7ae 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,77 +1,59 @@ -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 (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] - BroadcastHashJoin [ctr_customer_sk,c_customer_sk] - Project [ctr_customer_sk,ctr_total_return] - BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] - Filter [ctr_total_return] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [wr_returning_customer_sk,ca_state,sum,wr_return_amt] - CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] - CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] - CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [ca_address_sk,ca_state] #4 - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - 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] #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] #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometHashAggregate [wr_returning_customer_sk,ca_state,sum,wr_return_amt] - CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] - CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] - CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_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 - ReusedExchange [d_date_sk] #3 - ReusedExchange [ca_address_sk,ca_state] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometFilter [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] - 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 #9 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [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] + CometProject [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] + CometBroadcastHashJoin [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,ca_address_sk] + CometProject [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] + CometBroadcastHashJoin [ctr_customer_sk,ctr_total_return,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] + CometProject [ctr_customer_sk,ctr_total_return] + CometBroadcastHashJoin [ctr_customer_sk,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_state] + CometFilter [ctr_customer_sk,ctr_state,ctr_total_return] + CometHashAggregate [ctr_customer_sk,ctr_state,ctr_total_return,wr_returning_customer_sk,ca_state,sum,sum(UnscaledValue(wr_return_amt))] + CometColumnarExchange [wr_returning_customer_sk,ca_state] #1 + CometHashAggregate [wr_returning_customer_sk,ca_state,sum,wr_return_amt] + CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] + CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] + CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [ca_address_sk,ca_state] #4 + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_state] #5 + CometFilter [(avg(ctr_total_return) * 1.2),ctr_state] + CometHashAggregate [(avg(ctr_total_return) * 1.2),ctr_state,ctr_state,sum,count,avg(ctr_total_return)] + CometColumnarExchange [ctr_state] #6 + CometHashAggregate [ctr_state,sum,count,ctr_total_return] + CometHashAggregate [ctr_state,ctr_total_return,wr_returning_customer_sk,ca_state,sum,sum(UnscaledValue(wr_return_amt))] + CometColumnarExchange [wr_returning_customer_sk,ca_state] #7 + CometHashAggregate [wr_returning_customer_sk,ca_state,sum,wr_return_amt] + CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] + CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] + CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_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 + ReusedExchange [d_date_sk] #3 + ReusedExchange [ca_address_sk,ca_state] #4 + CometBroadcastExchange [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] #8 + CometFilter [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] + 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] + CometBroadcastExchange [ca_address_sk] #9 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] 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 d63ab7356..7284a48b6 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,99 +1,94 @@ == Physical Plan == -* 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) +* ColumnarToRow (90) ++- CometSort (89) + +- CometColumnarExchange (88) + +- CometProject (87) + +- CometBroadcastHashJoin (86) + :- CometProject (73) + : +- CometBroadcastHashJoin (72) + : :- CometBroadcastHashJoin (59) + : : :- CometProject (46) + : : : +- CometBroadcastHashJoin (45) + : : : :- CometBroadcastHashJoin (30) + : : : : :- CometHashAggregate (15) + : : : : : +- CometColumnarExchange (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) + : : : : +- CometBroadcastExchange (29) + : : : : +- CometHashAggregate (28) + : : : : +- CometColumnarExchange (27) + : : : : +- CometHashAggregate (26) + : : : : +- CometProject (25) + : : : : +- CometBroadcastHashJoin (24) + : : : : :- CometProject (22) + : : : : : +- CometBroadcastHashJoin (21) + : : : : : :- CometFilter (17) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (16) + : : : : : +- CometBroadcastExchange (20) + : : : : : +- CometFilter (19) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (18) + : : : : +- ReusedExchange (23) + : : : +- CometBroadcastExchange (44) + : : : +- CometHashAggregate (43) + : : : +- CometColumnarExchange (42) + : : : +- CometHashAggregate (41) + : : : +- CometProject (40) + : : : +- CometBroadcastHashJoin (39) + : : : :- CometProject (37) + : : : : +- CometBroadcastHashJoin (36) + : : : : :- CometFilter (32) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (31) + : : : : +- CometBroadcastExchange (35) + : : : : +- CometFilter (34) + : : : : +- CometScan parquet spark_catalog.default.date_dim (33) + : : : +- ReusedExchange (38) + : : +- CometBroadcastExchange (58) + : : +- CometHashAggregate (57) + : : +- CometColumnarExchange (56) + : : +- CometHashAggregate (55) + : : +- CometProject (54) + : : +- CometBroadcastHashJoin (53) + : : :- CometProject (51) + : : : +- CometBroadcastHashJoin (50) + : : : :- CometFilter (48) + : : : : +- CometScan parquet spark_catalog.default.web_sales (47) + : : : +- ReusedExchange (49) + : : +- ReusedExchange (52) + : +- CometBroadcastExchange (71) + : +- CometHashAggregate (70) + : +- CometColumnarExchange (69) + : +- CometHashAggregate (68) + : +- CometProject (67) + : +- CometBroadcastHashJoin (66) + : :- CometProject (64) + : : +- CometBroadcastHashJoin (63) + : : :- CometFilter (61) + : : : +- CometScan parquet spark_catalog.default.web_sales (60) + : : +- ReusedExchange (62) + : +- ReusedExchange (65) + +- CometBroadcastExchange (85) + +- CometHashAggregate (84) + +- CometColumnarExchange (83) + +- CometHashAggregate (82) + +- CometProject (81) + +- CometBroadcastHashJoin (80) + :- CometProject (78) + : +- CometBroadcastHashJoin (77) + : :- CometFilter (75) + : : +- CometScan parquet spark_catalog.default.web_sales (74) + : +- ReusedExchange (76) + +- ReusedExchange (79) (1) Scan parquet spark_catalog.default.store_sales @@ -161,470 +156,438 @@ 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))] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(15) CometHashAggregate 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))#11] -Results [3]: [ca_county#9, d_year#6, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#11,17,2) AS store_sales#12] -(17) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#13, ss_ext_sales_price#14, ss_sold_date_sk#15] +(16) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13] 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#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#14)] PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct -(18) CometFilter -Input [3]: [ss_addr_sk#13, ss_ext_sales_price#14, ss_sold_date_sk#15] -Condition : isnotnull(ss_addr_sk#13) +(17) CometFilter +Input [3]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13] +Condition : isnotnull(ss_addr_sk#11) -(19) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_qoy#19] +(18) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#15, d_year#16, d_qoy#17] 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) 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)) - -(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] - -(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 - -(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] - -(24) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#20, ca_county#21] - -(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 - -(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] - -(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))] - -(28) ColumnarToRow [codegen id : 2] -Input [4]: [ca_county#21, d_qoy#19, d_year#18, sum#22] - -(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] - -(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] - -(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 - -(33) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] +(19) CometFilter +Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] +Condition : ((((isnotnull(d_qoy#17) AND isnotnull(d_year#16)) AND (d_qoy#17 = 2)) AND (d_year#16 = 2000)) AND isnotnull(d_date_sk#15)) + +(20) CometBroadcastExchange +Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] +Arguments: [d_date_sk#15, d_year#16, d_qoy#17] + +(21) CometBroadcastHashJoin +Left output [3]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13] +Right output [3]: [d_date_sk#15, d_year#16, d_qoy#17] +Arguments: [ss_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight + +(22) CometProject +Input [6]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_year#16, d_qoy#17] +Arguments: [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17], [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17] + +(23) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#18, ca_county#19] + +(24) CometBroadcastHashJoin +Left output [4]: [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17] +Right output [2]: [ca_address_sk#18, ca_county#19] +Arguments: [ss_addr_sk#11], [ca_address_sk#18], Inner, BuildRight + +(25) CometProject +Input [6]: [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_address_sk#18, ca_county#19] +Arguments: [ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_county#19], [ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_county#19] + +(26) CometHashAggregate +Input [4]: [ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_county#19] +Keys [3]: [ca_county#19, d_qoy#17, d_year#16] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#12))] + +(27) CometColumnarExchange +Input [4]: [ca_county#19, d_qoy#17, d_year#16, sum#20] +Arguments: hashpartitioning(ca_county#19, d_qoy#17, d_year#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(28) CometHashAggregate +Input [4]: [ca_county#19, d_qoy#17, d_year#16, sum#20] +Keys [3]: [ca_county#19, d_qoy#17, d_year#16] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#12))] + +(29) CometBroadcastExchange +Input [2]: [ca_county#19, store_sales#21] +Arguments: [ca_county#19, store_sales#21] + +(30) CometBroadcastHashJoin +Left output [3]: [ca_county#9, d_year#6, store_sales#22] +Right output [2]: [ca_county#19, store_sales#21] +Arguments: [ca_county#9], [ca_county#19], Inner, BuildRight + +(31) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25] 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#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct -(34) CometFilter -Input [3]: [ss_addr_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_addr_sk#24) +(32) CometFilter +Input [3]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25] +Condition : isnotnull(ss_addr_sk#23) -(35) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#28, d_year#29, d_qoy#30] +(33) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#27, d_year#28, d_qoy#29] 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 -(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)) +(34) CometFilter +Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] +Condition : ((((isnotnull(d_qoy#29) AND isnotnull(d_year#28)) AND (d_qoy#29 = 3)) AND (d_year#28 = 2000)) AND isnotnull(d_date_sk#27)) + +(35) CometBroadcastExchange +Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] +Arguments: [d_date_sk#27, d_year#28, d_qoy#29] -(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] +(36) CometBroadcastHashJoin +Left output [3]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25] +Right output [3]: [d_date_sk#27, d_year#28, d_qoy#29] +Arguments: [ss_sold_date_sk#25], [d_date_sk#27], Inner, BuildRight -(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 +(37) CometProject +Input [6]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25, d_date_sk#27, d_year#28, d_qoy#29] +Arguments: [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29], [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29] -(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] +(38) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#30, ca_county#31] -(40) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#31, ca_county#32] +(39) CometBroadcastHashJoin +Left output [4]: [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29] +Right output [2]: [ca_address_sk#30, ca_county#31] +Arguments: [ss_addr_sk#23], [ca_address_sk#30], Inner, BuildRight -(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 +(40) CometProject +Input [6]: [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_address_sk#30, ca_county#31] +Arguments: [ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_county#31], [ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_county#31] -(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] +(41) CometHashAggregate +Input [4]: [ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_county#31] +Keys [3]: [ca_county#31, d_qoy#29, d_year#28] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#24))] + +(42) CometColumnarExchange +Input [4]: [ca_county#31, d_qoy#29, d_year#28, sum#32] +Arguments: hashpartitioning(ca_county#31, d_qoy#29, d_year#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (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 - -(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] - -(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] +Input [4]: [ca_county#31, d_qoy#29, d_year#28, sum#32] +Keys [3]: [ca_county#31, d_qoy#29, d_year#28] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#24))] + +(44) CometBroadcastExchange +Input [2]: [ca_county#31, store_sales#33] +Arguments: [ca_county#31, store_sales#33] + +(45) CometBroadcastHashJoin +Left output [5]: [ca_county#9, d_year#6, store_sales#22, ca_county#19, store_sales#21] +Right output [2]: [ca_county#31, store_sales#33] +Arguments: [ca_county#19], [ca_county#31], Inner, BuildRight + +(46) CometProject +Input [7]: [ca_county#9, d_year#6, store_sales#22, ca_county#19, store_sales#21, ca_county#31, store_sales#33] +Arguments: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33], [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33] + +(47) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#37), dynamicpruningexpression(ws_sold_date_sk#37 IN dynamicpruning#38)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#37)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(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) +(48) CometFilter +Input [3]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Condition : isnotnull(ws_bill_addr_sk#34) + +(49) ReusedExchange [Reuses operator id: 5] +Output [3]: [d_date_sk#38, d_year#39, d_qoy#40] + +(50) CometBroadcastHashJoin +Left output [3]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Right output [3]: [d_date_sk#38, d_year#39, d_qoy#40] +Arguments: [ws_sold_date_sk#36], [d_date_sk#38], Inner, BuildRight -(52) ReusedExchange [Reuses operator id: 5] -Output [3]: [d_date_sk#39, d_year#40, d_qoy#41] +(51) CometProject +Input [6]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#38, d_year#39, d_qoy#40] +Arguments: [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40], [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40] + +(52) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#41, ca_county#42] (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 +Left output [4]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40] +Right output [2]: [ca_address_sk#41, ca_county#42] +Arguments: [ws_bill_addr_sk#34], [ca_address_sk#41], Inner, BuildRight (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] - -(55) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#42, ca_county#43] - -(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 - -(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] - -(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#43] -Join type: Inner -Join condition: None - -(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] +Input [6]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_address_sk#41, ca_county#42] +Arguments: [ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_county#42], [ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_county#42] + +(55) CometHashAggregate +Input [4]: [ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_county#42] +Keys [3]: [ca_county#42, d_qoy#40, d_year#39] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] + +(56) CometColumnarExchange +Input [4]: [ca_county#42, d_qoy#40, d_year#39, sum#43] +Arguments: hashpartitioning(ca_county#42, d_qoy#40, d_year#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(57) CometHashAggregate +Input [4]: [ca_county#42, d_qoy#40, d_year#39, sum#43] +Keys [3]: [ca_county#42, d_qoy#40, d_year#39] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] + +(58) CometBroadcastExchange +Input [2]: [ca_county#42, web_sales#44] +Arguments: [ca_county#42, web_sales#44] + +(59) CometBroadcastHashJoin +Left output [5]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33] +Right output [2]: [ca_county#42, web_sales#44] +Arguments: [ca_county#9], [ca_county#42], Inner, BuildRight + +(60) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#49), dynamicpruningexpression(ws_sold_date_sk#49 IN dynamicpruning#50)] +PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#48)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(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) - -(66) ReusedExchange [Reuses operator id: 21] -Output [3]: [d_date_sk#51, d_year#52, d_qoy#53] - -(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 - -(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] - -(69) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#54, ca_county#55] - -(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 - -(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] - -(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#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) - -(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] - -(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] +(61) CometFilter +Input [3]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47] +Condition : isnotnull(ws_bill_addr_sk#45) + +(62) ReusedExchange [Reuses operator id: 20] +Output [3]: [d_date_sk#49, d_year#50, d_qoy#51] + +(63) CometBroadcastHashJoin +Left output [3]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47] +Right output [3]: [d_date_sk#49, d_year#50, d_qoy#51] +Arguments: [ws_sold_date_sk#47], [d_date_sk#49], Inner, BuildRight + +(64) CometProject +Input [6]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47, d_date_sk#49, d_year#50, d_qoy#51] +Arguments: [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51], [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51] + +(65) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#52, ca_county#53] + +(66) CometBroadcastHashJoin +Left output [4]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51] +Right output [2]: [ca_address_sk#52, ca_county#53] +Arguments: [ws_bill_addr_sk#45], [ca_address_sk#52], Inner, BuildRight + +(67) CometProject +Input [6]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_address_sk#52, ca_county#53] +Arguments: [ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_county#53], [ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_county#53] + +(68) CometHashAggregate +Input [4]: [ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_county#53] +Keys [3]: [ca_county#53, d_qoy#51, d_year#50] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#46))] + +(69) CometColumnarExchange +Input [4]: [ca_county#53, d_qoy#51, d_year#50, sum#54] +Arguments: hashpartitioning(ca_county#53, d_qoy#51, d_year#50, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(70) CometHashAggregate +Input [4]: [ca_county#53, d_qoy#51, d_year#50, sum#54] +Keys [3]: [ca_county#53, d_qoy#51, d_year#50] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#46))] + +(71) CometBroadcastExchange +Input [2]: [ca_county#53, web_sales#55] +Arguments: [ca_county#53, web_sales#55] + +(72) CometBroadcastHashJoin +Left output [7]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44] +Right output [2]: [ca_county#53, web_sales#55] +Arguments: [ca_county#42], [ca_county#53], Inner, (CASE WHEN (web_sales#44 > 0.00) THEN (web_sales#55 / web_sales#44) END > CASE WHEN (store_sales#22 > 0.00) THEN (store_sales#21 / store_sales#22) END), BuildRight + +(73) CometProject +Input [9]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, ca_county#53, web_sales#55] +Arguments: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55], [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55] + +(74) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#60), dynamicpruningexpression(ws_sold_date_sk#60 IN dynamicpruning#61)] +PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#59)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(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) +(75) CometFilter +Input [3]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58] +Condition : isnotnull(ws_bill_addr_sk#56) -(81) ReusedExchange [Reuses operator id: 37] -Output [3]: [d_date_sk#62, d_year#63, d_qoy#64] +(76) ReusedExchange [Reuses operator id: 35] +Output [3]: [d_date_sk#60, d_year#61, d_qoy#62] -(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 +(77) CometBroadcastHashJoin +Left output [3]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58] +Right output [3]: [d_date_sk#60, d_year#61, d_qoy#62] +Arguments: [ws_sold_date_sk#58], [d_date_sk#60], Inner, BuildRight -(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] +(78) CometProject +Input [6]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58, d_date_sk#60, d_year#61, d_qoy#62] +Arguments: [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62], [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62] -(84) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#65, ca_county#66] +(79) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#63, ca_county#64] -(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 +(80) CometBroadcastHashJoin +Left output [4]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62] +Right output [2]: [ca_address_sk#63, ca_county#64] +Arguments: [ws_bill_addr_sk#56], [ca_address_sk#63], Inner, BuildRight -(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] +(81) CometProject +Input [6]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_address_sk#63, ca_county#64] +Arguments: [ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_county#64], [ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_county#64] -(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))] +(82) CometHashAggregate +Input [4]: [ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_county#64] +Keys [3]: [ca_county#64, d_qoy#62, d_year#61] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#57))] -(88) ColumnarToRow [codegen id : 10] -Input [4]: [ca_county#66, d_qoy#64, d_year#63, sum#67] +(83) CometColumnarExchange +Input [4]: [ca_county#64, d_qoy#62, d_year#61, sum#65] +Arguments: hashpartitioning(ca_county#64, d_qoy#62, d_year#61, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(89) Exchange -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] +(84) CometHashAggregate +Input [4]: [ca_county#64, d_qoy#62, d_year#61, sum#65] +Keys [3]: [ca_county#64, d_qoy#62, d_year#61] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#57))] -(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] +(85) CometBroadcastExchange +Input [2]: [ca_county#64, web_sales#66] +Arguments: [ca_county#64, web_sales#66] -(91) BroadcastExchange -Input [2]: [ca_county#66, web_sales#68] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] +(86) CometBroadcastHashJoin +Left output [8]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55] +Right output [2]: [ca_county#64, web_sales#66] +Arguments: [ca_county#42], [ca_county#64], Inner, (CASE WHEN (web_sales#55 > 0.00) THEN (web_sales#66 / web_sales#55) END > CASE WHEN (store_sales#21 > 0.00) THEN (store_sales#33 / store_sales#21) END), BuildRight -(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) +(87) CometProject +Input [10]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55, ca_county#64, web_sales#66] +Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70], [ca_county#9, d_year#6, (web_sales#55 / web_sales#44) AS web_q1_q2_increase#67, (store_sales#21 / store_sales#22) AS store_q1_q2_increase#68, (web_sales#66 / web_sales#55) AS web_q2_q3_increase#69, (store_sales#33 / store_sales#21) AS store_q2_q3_increase#70] -(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] +(88) CometColumnarExchange +Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] +Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(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] +(89) CometSort +Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] +Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70], [ca_county#9 ASC NULLS FIRST] -(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 +(90) ColumnarToRow [codegen id : 1] +Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (99) -+- * ColumnarToRow (98) - +- CometFilter (97) - +- CometScan parquet spark_catalog.default.date_dim (96) +BroadcastExchange (94) ++- * ColumnarToRow (93) + +- CometFilter (92) + +- CometScan parquet spark_catalog.default.date_dim (91) -(96) Scan parquet spark_catalog.default.date_dim +(91) 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 -(97) CometFilter +(92) 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)) -(98) ColumnarToRow [codegen id : 1] +(93) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -(99) BroadcastExchange +(94) 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=13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -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) +Subquery:2 Hosting operator id = 16 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 +BroadcastExchange (98) ++- * ColumnarToRow (97) + +- CometFilter (96) + +- CometScan parquet spark_catalog.default.date_dim (95) -(100) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_qoy#19] +(95) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#15, d_year#16, d_qoy#17] 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 -(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)) +(96) CometFilter +Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] +Condition : ((((isnotnull(d_qoy#17) AND isnotnull(d_year#16)) AND (d_qoy#17 = 2)) AND (d_year#16 = 2000)) AND isnotnull(d_date_sk#15)) -(102) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#17, d_year#18, d_qoy#19] +(97) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] -(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] +(98) BroadcastExchange +Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -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) +Subquery:3 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 +BroadcastExchange (102) ++- * ColumnarToRow (101) + +- CometFilter (100) + +- CometScan parquet spark_catalog.default.date_dim (99) -(104) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#28, d_year#29, d_qoy#30] +(99) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#27, d_year#28, d_qoy#29] 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 -(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)) +(100) CometFilter +Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] +Condition : ((((isnotnull(d_qoy#29) AND isnotnull(d_year#28)) AND (d_qoy#29 = 3)) AND (d_year#28 = 2000)) AND isnotnull(d_date_sk#27)) -(106) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#28, d_year#29, d_qoy#30] +(101) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] -(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] +(102) BroadcastExchange +Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -Subquery:4 Hosting operator id = 50 Hosting Expression = ws_sold_date_sk#37 IN dynamicpruning#4 +Subquery:4 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#4 -Subquery:5 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#49 IN dynamicpruning#16 +Subquery:5 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#14 -Subquery:6 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#60 IN dynamicpruning#27 +Subquery:6 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#26 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 a48c7ad26..6dcf5714d 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,150 +1,116 @@ -WholeStageCodegen (13) - Sort [ca_county] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [ca_county] #1 - 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] - BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county] - Project [ca_county,d_year,store_sales,store_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county] - BroadcastHashJoin [ca_county,ca_county] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] - CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] - CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #4 - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [ca_address_sk,ca_county] #5 - CometFilter [ca_address_sk,ca_county] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - InputAdapter - 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] #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] - CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] - CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_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 #8 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #9 + CometSort [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] + CometColumnarExchange [ca_county] #1 + CometProject [web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] + CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales,ca_county,web_sales] + CometProject [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] + CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,ca_county,web_sales] + CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales] + CometProject [ca_county,d_year,store_sales,store_sales,store_sales] + CometBroadcastHashJoin [ca_county,d_year,store_sales,ca_county,store_sales,ca_county,store_sales] + CometBroadcastHashJoin [ca_county,d_year,store_sales,ca_county,store_sales] + CometHashAggregate [ca_county,d_year,store_sales,d_qoy,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange [ca_county,d_qoy,d_year] #2 + CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] + CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #4 + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [ca_address_sk,ca_county] #5 + CometFilter [ca_address_sk,ca_county] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + CometBroadcastExchange [ca_county,store_sales] #6 + CometHashAggregate [ca_county,store_sales,d_qoy,d_year,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange [ca_county,d_qoy,d_year] #7 + CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] + CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_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 #8 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - ReusedExchange [ca_address_sk,ca_county] #5 - InputAdapter - 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] #11 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] - CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] - CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_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 #12 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #13 + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #9 + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + ReusedExchange [ca_address_sk,ca_county] #5 + CometBroadcastExchange [ca_county,store_sales] #10 + CometHashAggregate [ca_county,store_sales,d_qoy,d_year,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange [ca_county,d_qoy,d_year] #11 + CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] + CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_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 #12 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - ReusedExchange [ca_address_sk,ca_county] #5 - InputAdapter - 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] #15 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] - CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] - CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 - ReusedExchange [d_date_sk,d_year,d_qoy] #4 - ReusedExchange [ca_address_sk,ca_county] #5 - InputAdapter - 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] #17 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] - CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] - CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 - ReusedExchange [d_date_sk,d_year,d_qoy] #9 - ReusedExchange [ca_address_sk,ca_county] #5 - InputAdapter - 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] #19 - WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] - CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] - CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 - ReusedExchange [d_date_sk,d_year,d_qoy] #13 - ReusedExchange [ca_address_sk,ca_county] #5 + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #13 + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + ReusedExchange [ca_address_sk,ca_county] #5 + CometBroadcastExchange [ca_county,web_sales] #14 + CometHashAggregate [ca_county,web_sales,d_qoy,d_year,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometColumnarExchange [ca_county,d_qoy,d_year] #15 + CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] + CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 + ReusedExchange [d_date_sk,d_year,d_qoy] #4 + ReusedExchange [ca_address_sk,ca_county] #5 + CometBroadcastExchange [ca_county,web_sales] #16 + CometHashAggregate [ca_county,web_sales,d_qoy,d_year,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometColumnarExchange [ca_county,d_qoy,d_year] #17 + CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] + CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 + ReusedExchange [d_date_sk,d_year,d_qoy] #9 + ReusedExchange [ca_address_sk,ca_county] #5 + CometBroadcastExchange [ca_county,web_sales] #18 + CometHashAggregate [ca_county,web_sales,d_qoy,d_year,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometColumnarExchange [ca_county,d_qoy,d_year] #19 + CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] + CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 + 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 b7eee60b1..701a75e24 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,35 +1,34 @@ == Physical Plan == -* 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) +* ColumnarToRow (30) ++- CometHashAggregate (29) + +- CometColumnarExchange (28) + +- CometHashAggregate (27) + +- CometProject (26) + +- CometBroadcastHashJoin (25) + :- CometProject (23) + : +- CometBroadcastHashJoin (22) + : :- 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 (21) + : +- CometFilter (20) + : +- CometHashAggregate (19) + : +- CometColumnarExchange (18) + : +- CometHashAggregate (17) + : +- CometProject (16) + : +- CometBroadcastHashJoin (15) + : :- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.catalog_sales (9) + : +- CometBroadcastExchange (14) + : +- CometProject (13) + : +- CometFilter (12) + : +- CometScan parquet spark_catalog.default.date_dim (11) + +- ReusedExchange (24) (1) Scan parquet spark_catalog.default.catalog_sales @@ -72,10 +71,7 @@ Arguments: [cs_item_sk#1], [i_item_sk#5], Inner, BuildRight 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] - -(10) Scan parquet spark_catalog.default.catalog_sales +(9) 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 [] @@ -83,138 +79,130 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(11) CometFilter +(10) CometFilter Input [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] Condition : isnotnull(cs_item_sk#7) -(12) Scan parquet spark_catalog.default.date_dim +(11) 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 -(13) CometFilter +(12) 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)) -(14) CometProject +(13) CometProject Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(15) CometBroadcastExchange +(14) CometBroadcastExchange Input [1]: [d_date_sk#11] Arguments: [d_date_sk#11] -(16) CometBroadcastHashJoin +(15) 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 +(16) 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) CometHashAggregate +(17) 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))] -(19) ColumnarToRow [codegen id : 1] -Input [3]: [cs_item_sk#7, sum#13, count#14] - -(20) Exchange +(18) CometColumnarExchange Input [3]: [cs_item_sk#7, sum#13, count#14] -Arguments: hashpartitioning(cs_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: hashpartitioning(cs_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(21) HashAggregate [codegen id : 2] +(19) CometHashAggregate 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))#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] -(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) +(20) CometFilter +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] +Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#15) -(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] +(21) CometBroadcastExchange +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] +Arguments: [(1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] -(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))#16) +(22) CometBroadcastHashJoin +Left output [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] +Right output [2]: [(1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] +Arguments: [i_item_sk#5], [cs_item_sk#7], Inner, (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#15), BuildRight -(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))#16, cs_item_sk#7] +(23) CometProject +Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] +Arguments: [cs_ext_discount_amt#2, cs_sold_date_sk#3], [cs_ext_discount_amt#2, cs_sold_date_sk#3] -(26) ReusedExchange [Reuses operator id: 36] -Output [1]: [d_date_sk#17] +(24) ReusedExchange [Reuses operator id: 14] +Output [1]: [d_date_sk#16] -(27) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#17] -Join type: Inner -Join condition: None +(25) CometBroadcastHashJoin +Left output [2]: [cs_ext_discount_amt#2, cs_sold_date_sk#3] +Right output [1]: [d_date_sk#16] +Arguments: [cs_sold_date_sk#3], [d_date_sk#16], Inner, BuildRight -(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#17] +(26) CometProject +Input [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, d_date_sk#16] +Arguments: [cs_ext_discount_amt#2], [cs_ext_discount_amt#2] -(29) HashAggregate [codegen id : 4] +(27) CometHashAggregate Input [1]: [cs_ext_discount_amt#2] Keys: [] Functions [1]: [partial_sum(UnscaledValue(cs_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum#18] -Results [1]: [sum#19] -(30) Exchange -Input [1]: [sum#19] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] +(28) CometColumnarExchange +Input [1]: [sum#17] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(31) HashAggregate [codegen id : 5] -Input [1]: [sum#19] +(29) CometHashAggregate +Input [1]: [sum#17] Keys: [] Functions [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))] -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] + +(30) ColumnarToRow [codegen id : 1] +Input [1]: [excess discount amount#18] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (36) -+- * ColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan parquet spark_catalog.default.date_dim (32) +BroadcastExchange (35) ++- * ColumnarToRow (34) + +- CometProject (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.date_dim (31) -(32) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#17, d_date#22] +(31) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#16, d_date#19] 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 -(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) CometFilter +Input [2]: [d_date_sk#16, d_date#19] +Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-01-27)) AND (d_date#19 <= 2000-04-26)) AND isnotnull(d_date_sk#16)) -(34) CometProject -Input [2]: [d_date_sk#17, d_date#22] -Arguments: [d_date_sk#17], [d_date_sk#17] +(33) CometProject +Input [2]: [d_date_sk#16, d_date#19] +Arguments: [d_date_sk#16], [d_date_sk#16] -(35) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#17] +(34) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#16] -(36) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(35) BroadcastExchange +Input [1]: [d_date_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -Subquery:2 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 9 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 c3bfdfd77..19be8718e 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,50 +1,41 @@ -WholeStageCodegen (5) - HashAggregate [sum] [sum(UnscaledValue(cs_ext_discount_amt)),excess discount amount,sum] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange #1 - 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))] - ColumnarToRow - InputAdapter - CometProject [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] - CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] - CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk] #3 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - InputAdapter - BroadcastExchange #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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [cs_item_sk,sum,count,cs_ext_discount_amt] - CometProject [cs_item_sk,cs_ext_discount_amt] - CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,d_date_sk] - CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - ReusedExchange [d_date_sk] #2 + CometHashAggregate [excess discount amount,sum,sum(UnscaledValue(cs_ext_discount_amt))] + CometColumnarExchange #1 + CometHashAggregate [sum,cs_ext_discount_amt] + CometProject [cs_ext_discount_amt] + CometBroadcastHashJoin [cs_ext_discount_amt,cs_sold_date_sk,d_date_sk] + CometProject [cs_ext_discount_amt,cs_sold_date_sk] + CometBroadcastHashJoin [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk,(1.3 * avg(cs_ext_discount_amt)),cs_item_sk] + CometProject [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] + CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk] #3 + CometProject [i_item_sk] + CometFilter [i_item_sk,i_manufact_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + CometBroadcastExchange [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk] #4 + CometFilter [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk] + CometHashAggregate [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk,sum,count,avg(UnscaledValue(cs_ext_discount_amt))] + CometColumnarExchange [cs_item_sk] #5 + CometHashAggregate [cs_item_sk,sum,count,cs_ext_discount_amt] + CometProject [cs_item_sk,cs_ext_discount_amt] + CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,d_date_sk] + CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + ReusedExchange [d_date_sk] #6 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 8e385ed4e..6fb8b05fb 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 @@ -1,67 +1,65 @@ == Physical Plan == -TakeOrderedAndProject (63) -+- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- Union (59) - :- * HashAggregate (28) - : +- Exchange (27) - : +- * 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) - : +- * 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) - +- * 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) +* ColumnarToRow (61) ++- CometTakeOrderedAndProject (60) + +- CometHashAggregate (59) + +- CometColumnarExchange (58) + +- CometHashAggregate (57) + +- CometUnion (56) + :- CometHashAggregate (27) + : +- CometColumnarExchange (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) + :- CometHashAggregate (41) + : +- CometColumnarExchange (40) + : +- CometHashAggregate (39) + : +- CometProject (38) + : +- CometBroadcastHashJoin (37) + : :- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometProject (32) + : : : +- CometBroadcastHashJoin (31) + : : : :- CometFilter (29) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (28) + : : : +- ReusedExchange (30) + : : +- ReusedExchange (33) + : +- ReusedExchange (36) + +- CometHashAggregate (55) + +- CometColumnarExchange (54) + +- CometHashAggregate (53) + +- CometProject (52) + +- CometBroadcastHashJoin (51) + :- CometProject (49) + : +- CometBroadcastHashJoin (48) + : :- CometProject (46) + : : +- CometBroadcastHashJoin (45) + : : :- CometFilter (43) + : : : +- CometScan parquet spark_catalog.default.web_sales (42) + : : +- ReusedExchange (44) + : +- ReusedExchange (47) + +- ReusedExchange (50) (1) Scan parquet spark_catalog.default.store_sales @@ -185,212 +183,199 @@ 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))] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarExchange Input [2]: [i_manufact_id#12, sum#15] +Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(27) Exchange -Input [2]: [i_manufact_id#12, sum#15] -Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=1] - -(28) HashAggregate [codegen id : 2] +(27) CometHashAggregate 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))#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#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] +(28) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_bill_addr_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#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#22)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#20)] PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(30) CometFilter -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)) +(29) CometFilter +Input [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] +Condition : (isnotnull(cs_bill_addr_sk#16) AND isnotnull(cs_item_sk#17)) -(31) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#23] +(30) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#21] -(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 +(31) CometBroadcastHashJoin +Left output [4]: [cs_bill_addr_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 -(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] +(32) CometProject +Input [5]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19, d_date_sk#21] +Arguments: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18], [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18] -(34) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#24] +(33) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#22] -(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 +(34) CometBroadcastHashJoin +Left output [3]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18] +Right output [1]: [ca_address_sk#22] +Arguments: [cs_bill_addr_sk#16], [ca_address_sk#22], 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] +(35) CometProject +Input [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, ca_address_sk#22] +Arguments: [cs_item_sk#17, cs_ext_sales_price#18], [cs_item_sk#17, cs_ext_sales_price#18] -(37) ReusedExchange [Reuses operator id: 22] -Output [2]: [i_item_sk#25, i_manufact_id#26] +(36) ReusedExchange [Reuses operator id: 22] +Output [2]: [i_item_sk#23, i_manufact_id#24] -(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 +(37) CometBroadcastHashJoin +Left output [2]: [cs_item_sk#17, cs_ext_sales_price#18] +Right output [2]: [i_item_sk#23, i_manufact_id#24] +Arguments: [cs_item_sk#17], [i_item_sk#23], 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] +(38) CometProject +Input [4]: [cs_item_sk#17, cs_ext_sales_price#18, i_item_sk#23, i_manufact_id#24] +Arguments: [cs_ext_sales_price#18, i_manufact_id#24], [cs_ext_sales_price#18, i_manufact_id#24] -(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))] +(39) CometHashAggregate +Input [2]: [cs_ext_sales_price#18, i_manufact_id#24] +Keys [1]: [i_manufact_id#24] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#18))] -(41) ColumnarToRow [codegen id : 3] -Input [2]: [i_manufact_id#26, sum#27] +(40) CometColumnarExchange +Input [2]: [i_manufact_id#24, sum#25] +Arguments: hashpartitioning(i_manufact_id#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(42) Exchange -Input [2]: [i_manufact_id#26, sum#27] -Arguments: hashpartitioning(i_manufact_id#26, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(41) CometHashAggregate +Input [2]: [i_manufact_id#24, sum#25] +Keys [1]: [i_manufact_id#24] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#18))] -(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] +(42) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] 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#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#30)] 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)) +(43) CometFilter +Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] +Condition : (isnotnull(ws_bill_addr_sk#27) AND isnotnull(ws_item_sk#26)) -(46) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#35] +(44) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#31] -(47) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] -Right output [1]: [d_date_sk#35] -Arguments: [ws_sold_date_sk#33], [d_date_sk#35], Inner, BuildRight +(45) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] +Right output [1]: [d_date_sk#31] +Arguments: [ws_sold_date_sk#29], [d_date_sk#31], Inner, BuildRight -(48) CometProject -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] -Arguments: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32], [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] +(46) CometProject +Input [5]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29, d_date_sk#31] +Arguments: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28], [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28] -(49) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#36] +(47) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#32] -(50) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] -Right output [1]: [ca_address_sk#36] -Arguments: [ws_bill_addr_sk#31], [ca_address_sk#36], Inner, BuildRight +(48) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28] +Right output [1]: [ca_address_sk#32] +Arguments: [ws_bill_addr_sk#27], [ca_address_sk#32], Inner, BuildRight -(51) CometProject -Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ca_address_sk#36] -Arguments: [ws_item_sk#30, ws_ext_sales_price#32], [ws_item_sk#30, ws_ext_sales_price#32] +(49) CometProject +Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ca_address_sk#32] +Arguments: [ws_item_sk#26, ws_ext_sales_price#28], [ws_item_sk#26, ws_ext_sales_price#28] -(52) ReusedExchange [Reuses operator id: 22] -Output [2]: [i_item_sk#37, i_manufact_id#38] +(50) ReusedExchange [Reuses operator id: 22] +Output [2]: [i_item_sk#33, i_manufact_id#34] -(53) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_ext_sales_price#32] -Right output [2]: [i_item_sk#37, i_manufact_id#38] -Arguments: [ws_item_sk#30], [i_item_sk#37], Inner, BuildRight +(51) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#26, ws_ext_sales_price#28] +Right output [2]: [i_item_sk#33, i_manufact_id#34] +Arguments: [ws_item_sk#26], [i_item_sk#33], Inner, BuildRight -(54) CometProject -Input [4]: [ws_item_sk#30, ws_ext_sales_price#32, i_item_sk#37, i_manufact_id#38] -Arguments: [ws_ext_sales_price#32, i_manufact_id#38], [ws_ext_sales_price#32, i_manufact_id#38] +(52) CometProject +Input [4]: [ws_item_sk#26, ws_ext_sales_price#28, i_item_sk#33, i_manufact_id#34] +Arguments: [ws_ext_sales_price#28, i_manufact_id#34], [ws_ext_sales_price#28, i_manufact_id#34] -(55) CometHashAggregate -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))] +(53) CometHashAggregate +Input [2]: [ws_ext_sales_price#28, i_manufact_id#34] +Keys [1]: [i_manufact_id#34] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#28))] -(56) ColumnarToRow [codegen id : 5] -Input [2]: [i_manufact_id#38, sum#39] +(54) CometColumnarExchange +Input [2]: [i_manufact_id#34, sum#35] +Arguments: hashpartitioning(i_manufact_id#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(57) Exchange -Input [2]: [i_manufact_id#38, sum#39] -Arguments: hashpartitioning(i_manufact_id#38, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(58) HashAggregate [codegen id : 6] -Input [2]: [i_manufact_id#38, sum#39] -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))#40] -Results [2]: [i_manufact_id#38, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#32))#40,17,2) AS total_sales#41] +(55) CometHashAggregate +Input [2]: [i_manufact_id#34, sum#35] +Keys [1]: [i_manufact_id#34] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#28))] -(59) Union +(56) CometUnion +Child 0 Input [2]: [i_manufact_id#12, total_sales#36] +Child 1 Input [2]: [i_manufact_id#24, total_sales#37] +Child 2 Input [2]: [i_manufact_id#34, total_sales#38] -(60) HashAggregate [codegen id : 7] -Input [2]: [i_manufact_id#12, total_sales#17] +(57) CometHashAggregate +Input [2]: [i_manufact_id#12, total_sales#36] Keys [1]: [i_manufact_id#12] -Functions [1]: [partial_sum(total_sales#17)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [3]: [i_manufact_id#12, sum#44, isEmpty#45] +Functions [1]: [partial_sum(total_sales#36)] -(61) Exchange -Input [3]: [i_manufact_id#12, sum#44, isEmpty#45] -Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(58) CometColumnarExchange +Input [3]: [i_manufact_id#12, sum#39, isEmpty#40] +Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(62) HashAggregate [codegen id : 8] -Input [3]: [i_manufact_id#12, sum#44, isEmpty#45] +(59) CometHashAggregate +Input [3]: [i_manufact_id#12, sum#39, isEmpty#40] Keys [1]: [i_manufact_id#12] -Functions [1]: [sum(total_sales#17)] -Aggregate Attributes [1]: [sum(total_sales#17)#46] -Results [2]: [i_manufact_id#12, sum(total_sales#17)#46 AS total_sales#47] +Functions [1]: [sum(total_sales#36)] + +(60) CometTakeOrderedAndProject +Input [2]: [i_manufact_id#12, total_sales#41] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_sales#41 ASC NULLS FIRST], output=[i_manufact_id#12,total_sales#41]), [i_manufact_id#12, total_sales#41], 100, [total_sales#41 ASC NULLS FIRST], [i_manufact_id#12, total_sales#41] -(63) TakeOrderedAndProject -Input [2]: [i_manufact_id#12, total_sales#47] -Arguments: 100, [total_sales#47 ASC NULLS FIRST], [i_manufact_id#12, total_sales#47] +(61) ColumnarToRow [codegen id : 1] +Input [2]: [i_manufact_id#12, total_sales#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (68) -+- * ColumnarToRow (67) - +- CometProject (66) - +- CometFilter (65) - +- CometScan parquet spark_catalog.default.date_dim (64) +BroadcastExchange (66) ++- * ColumnarToRow (65) + +- CometProject (64) + +- CometFilter (63) + +- CometScan parquet spark_catalog.default.date_dim (62) -(64) Scan parquet spark_catalog.default.date_dim +(62) Scan parquet spark_catalog.default.date_dim 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 +(63) 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)) -(66) CometProject +(64) CometProject 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] +(65) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(68) BroadcastExchange +(66) BroadcastExchange Input [1]: [d_date_sk#6] 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#21 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 28 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#29 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 7a3c2c153..964f250ba 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,89 +1,73 @@ -TakeOrderedAndProject [total_sales,i_manufact_id] - WholeStageCodegen (8) - HashAggregate [i_manufact_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - InputAdapter - Exchange [i_manufact_id] #1 - WholeStageCodegen (7) - HashAggregate [i_manufact_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (2) - HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_manufact_id] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_manufact_id,sum,ss_ext_sales_price] - CometProject [ss_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_manufact_id] - CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_manufact_id,total_sales] + CometHashAggregate [i_manufact_id,total_sales,sum,isEmpty,sum(total_sales)] + CometColumnarExchange [i_manufact_id] #1 + CometHashAggregate [i_manufact_id,sum,isEmpty,total_sales] + CometUnion [i_manufact_id,total_sales] + CometHashAggregate [i_manufact_id,total_sales,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange [i_manufact_id] #2 + CometHashAggregate [i_manufact_id,sum,ss_ext_sales_price] + CometProject [ss_ext_sales_price,i_manufact_id] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_manufact_id] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ca_address_sk] #5 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk,i_manufact_id] #6 - CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id] - CometFilter [i_item_sk,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange [i_manufact_id] #7 - CometProject [i_manufact_id] - CometFilter [i_category,i_manufact_id] - 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] #8 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometHashAggregate [i_manufact_id,sum,cs_ext_sales_price] - CometProject [cs_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_manufact_id] - CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] - CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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 - 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] #9 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometHashAggregate [i_manufact_id,sum,ws_ext_sales_price] - CometProject [ws_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_manufact_id] - CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] - CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_manufact_id] #6 + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ca_address_sk] #5 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometBroadcastExchange [i_item_sk,i_manufact_id] #6 + CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id] + CometFilter [i_item_sk,i_manufact_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + CometBroadcastExchange [i_manufact_id] #7 + CometProject [i_manufact_id] + CometFilter [i_category,i_manufact_id] + CometScan parquet spark_catalog.default.item [i_category,i_manufact_id] + CometHashAggregate [i_manufact_id,total_sales,sum,sum(UnscaledValue(cs_ext_sales_price))] + CometColumnarExchange [i_manufact_id] #8 + CometHashAggregate [i_manufact_id,sum,cs_ext_sales_price] + CometProject [cs_ext_sales_price,i_manufact_id] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_manufact_id] + CometProject [cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] + CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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 + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_manufact_id] #6 + CometHashAggregate [i_manufact_id,total_sales,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometColumnarExchange [i_manufact_id] #9 + CometHashAggregate [i_manufact_id,sum,ws_ext_sales_price] + CometProject [ws_ext_sales_price,i_manufact_id] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_manufact_id] + CometProject [ws_item_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] + CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 + 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 5609dc11c..c33735fa4 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,37 +1,36 @@ == Physical Plan == -* 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) +* ColumnarToRow (32) ++- CometSort (31) + +- CometColumnarExchange (30) + +- CometProject (29) + +- CometBroadcastHashJoin (28) + :- CometFilter (24) + : +- CometHashAggregate (23) + : +- CometColumnarExchange (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) + +- CometBroadcastExchange (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.customer (25) (1) Scan parquet spark_catalog.default.store_sales @@ -135,90 +134,84 @@ 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)] -(22) ColumnarToRow [codegen id : 1] +(22) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(24) HashAggregate [codegen id : 3] +(23) CometHashAggregate 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)#17] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#17 AS cnt#18] -(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)) +(24) CometFilter +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] +Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) -(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] +(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] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(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) +(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) -(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] +(27) CometBroadcastExchange +Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Arguments: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -(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] +(28) CometBroadcastHashJoin +Left output [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] +Right output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight -(30) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#19] -Join type: Inner -Join condition: None +(29) CometProject +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] +Arguments: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17], [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -(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] +(30) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] -(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] +(31) CometSort +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, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17], [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] -(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 +(32) ColumnarToRow [codegen id : 1] +Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (38) -+- * ColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.date_dim (34) +BroadcastExchange (37) ++- * ColumnarToRow (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) -(34) Scan parquet spark_catalog.default.date_dim +(33) 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 -(35) CometFilter +(34) 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)) -(36) CometProject +(35) CometProject Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(37) ColumnarToRow [codegen id : 1] +(36) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(38) BroadcastExchange +(37) BroadcastExchange Input [1]: [d_date_sk#7] -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/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt index e1f7f9bd2..c131bd9c0 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,50 +1,42 @@ -WholeStageCodegen (4) - Sort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,count] - CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] - CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] - CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #5 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange [hd_demo_sk] #6 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 + CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometFilter [ss_ticket_number,ss_customer_sk,cnt] + CometHashAggregate [ss_ticket_number,ss_customer_sk,cnt,count,count(1)] + CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] + CometProject [ss_customer_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange [s_store_sk] #5 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_county] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county] + CometBroadcastExchange [hd_demo_sk] #6 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] #7 + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] 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 c06c1dd16..b35338d18 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,48 +1,50 @@ == Physical Plan == -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) +TakeOrderedAndProject (46) ++- * HashAggregate (45) + +- * ColumnarToRow (44) + +- CometColumnarExchange (43) + +- RowToColumnar (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 @@ -238,50 +240,56 @@ Functions [10]: [partial_count(1), partial_min(cd_dep_count#25), partial_max(cd_ 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] -(42) Exchange +(42) RowToColumnar 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] -(43) HashAggregate [codegen id : 6] +(43) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=5] + +(44) ColumnarToRow [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] + +(45) 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] -(44) TakeOrderedAndProject +(46) 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 = 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) +BroadcastExchange (51) ++- * ColumnarToRow (50) + +- CometProject (49) + +- CometFilter (48) + +- CometScan parquet spark_catalog.default.date_dim (47) -(45) Scan parquet spark_catalog.default.date_dim +(47) 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 -(46) CometFilter +(48) 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)) -(47) CometProject +(49) CometProject Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(48) ColumnarToRow [codegen id : 1] +(50) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(49) BroadcastExchange +(51) BroadcastExchange Input [1]: [d_date_sk#9] 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/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt index 9deb51342..f9ef660d5 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,70 +1,72 @@ 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 (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 (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] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] - CometFilter [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] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,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 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + RowToColumnar + 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] + Project [c_current_cdemo_sk,ca_state] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] + Filter [exists,exists] + BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,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 + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [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] + CometBroadcastExchange [ss_customer_sk] #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,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 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [cs_ship_customer_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,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 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) + BroadcastExchange #8 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - 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] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + 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-v1_4/q36/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt index 33b572ff6..9f22e37d6 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 @@ -2,11 +2,11 @@ TakeOrderedAndProject (29) +- * Project (28) +- Window (27) - +- * Sort (26) - +- Exchange (25) - +- * HashAggregate (24) - +- Exchange (23) - +- * ColumnarToRow (22) + +- * ColumnarToRow (26) + +- CometSort (25) + +- CometColumnarExchange (24) + +- CometHashAggregate (23) + +- CometColumnarExchange (22) +- CometHashAggregate (21) +- CometExpand (20) +- CometProject (19) @@ -131,39 +131,37 @@ Input [5]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#14, i_class#15, sp 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))] -(22) ColumnarToRow [codegen id : 1] +(22) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(24) HashAggregate [codegen id : 2] +(23) CometHashAggregate 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))#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] -(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] +(24) CometColumnarExchange +Input [7]: [gross_margin#19, i_category#14, i_class#15, lochierarchy#20, _w0#21, _w1#22, _w2#23] +Arguments: hashpartitioning(_w1#22, _w2#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(25) CometSort +Input [7]: [gross_margin#19, i_category#14, i_class#15, lochierarchy#20, _w0#21, _w1#22, _w2#23] +Arguments: [gross_margin#19, i_category#14, i_class#15, lochierarchy#20, _w0#21, _w1#22, _w2#23], [_w1#22 ASC NULLS FIRST, _w2#23 ASC NULLS FIRST, _w0#21 ASC NULLS FIRST] -(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) ColumnarToRow [codegen id : 1] +Input [7]: [gross_margin#19, i_category#14, i_class#15, lochierarchy#20, _w0#21, _w1#22, _w2#23] (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] +Input [7]: [gross_margin#19, i_category#14, i_class#15, lochierarchy#20, _w0#21, _w1#22, _w2#23] +Arguments: [rank(_w0#21) windowspecdefinition(_w1#22, _w2#23, _w0#21 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#24], [_w1#22, _w2#23], [_w0#21 ASC NULLS FIRST] -(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) Project [codegen id : 2] +Output [5]: [gross_margin#19, i_category#14, i_class#15, lochierarchy#20, rank_within_parent#24] +Input [8]: [gross_margin#19, i_category#14, i_class#15, lochierarchy#20, _w0#21, _w1#22, _w2#23, rank_within_parent#24] (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] +Input [5]: [gross_margin#19, i_category#14, i_class#15, lochierarchy#20, rank_within_parent#24] +Arguments: 100, [lochierarchy#20 DESC NULLS LAST, CASE WHEN (lochierarchy#20 = 0) THEN i_category#14 END ASC NULLS FIRST, rank_within_parent#24 ASC NULLS FIRST], [gross_margin#19, i_category#14, i_class#15, lochierarchy#20, rank_within_parent#24] ===== Subqueries ===== 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 4ad9b501b..d7beb4422 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,45 +1,41 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (4) + WholeStageCodegen (2) Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [_w0,_w1,_w2] - WholeStageCodegen (3) - Sort [_w1,_w2,_w0] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [_w1,_w2] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_category,i_class,spark_grouping_id,sum,sum,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,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] - CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] - CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometFilter [i_item_sk,i_class,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #6 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] + CometColumnarExchange [_w1,_w2] #1 + CometHashAggregate [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2,spark_grouping_id,sum,sum,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange [i_category,i_class,spark_grouping_id] #2 + CometHashAggregate [i_category,i_class,spark_grouping_id,sum,sum,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,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] + CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometFilter [i_item_sk,i_class,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometBroadcastExchange [s_store_sk] #6 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + 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 6eb0f14b1..42dab0b38 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,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (26) -+- * HashAggregate (25) - +- Exchange (24) - +- * ColumnarToRow (23) +* ColumnarToRow (26) ++- CometTakeOrderedAndProject (25) + +- CometHashAggregate (24) + +- CometColumnarExchange (23) +- CometHashAggregate (22) +- CometProject (21) +- CometBroadcastHashJoin (20) @@ -132,23 +132,21 @@ 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: [] -(23) ColumnarToRow [codegen id : 1] +(23) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(24) 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] -(26) TakeOrderedAndProject +(25) CometTakeOrderedAndProject +Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#2 ASC NULLS FIRST], output=[i_item_id#2,i_item_desc#3,i_current_price#4]), [i_item_id#2, i_item_desc#3, i_current_price#4], 100, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4] + +(26) ColumnarToRow [codegen id : 1] 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 ===== 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 efd86d5ab..39edfb519 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,38 +1,36 @@ -TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - 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 (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,i_item_id,i_item_desc,i_current_price,cs_item_sk] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2 - CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,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,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] - CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 - CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_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_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] + CometHashAggregate [i_item_id,i_item_desc,i_current_price] + CometColumnarExchange [i_item_id,i_item_desc,i_current_price] #1 + CometHashAggregate [i_item_id,i_item_desc,i_current_price] + CometProject [i_item_id,i_item_desc,i_current_price] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,cs_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2 + CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,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,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] + CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] + CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 + CometProject [inv_item_sk,inv_date_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_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_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [cs_item_sk] + CometFilter [cs_item_sk,cs_sold_date_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 b6b11827d..091883bd6 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,53 +1,51 @@ == Physical Plan == -* 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) +* ColumnarToRow (47) ++- CometHashAggregate (46) + +- CometColumnarExchange (45) + +- CometHashAggregate (44) + +- CometProject (43) + +- CometBroadcastHashJoin (42) + :- CometBroadcastHashJoin (29) + : :- CometHashAggregate (16) + : : +- CometColumnarExchange (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) + : +- CometBroadcastExchange (28) + : +- CometHashAggregate (27) + : +- CometColumnarExchange (26) + : +- CometHashAggregate (25) + : +- CometProject (24) + : +- CometBroadcastHashJoin (23) + : :- CometProject (21) + : : +- CometBroadcastHashJoin (20) + : : :- CometFilter (18) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (17) + : : +- ReusedExchange (19) + : +- ReusedExchange (22) + +- CometBroadcastExchange (41) + +- CometHashAggregate (40) + +- CometColumnarExchange (39) + +- CometHashAggregate (38) + +- CometProject (37) + +- CometBroadcastHashJoin (36) + :- CometProject (34) + : +- CometBroadcastHashJoin (33) + : :- CometFilter (31) + : : +- CometScan parquet spark_catalog.default.web_sales (30) + : +- ReusedExchange (32) + +- ReusedExchange (35) (1) Scan parquet spark_catalog.default.store_sales @@ -119,21 +117,16 @@ 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: [] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(17) HashAggregate [codegen id : 6] +(16) 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#9, c_first_name#8, d_date#5] -(18) Scan parquet spark_catalog.default.catalog_sales +(17) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_bill_customer_sk#10, cs_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] @@ -141,64 +134,58 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#11), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(19) CometFilter +(18) CometFilter Input [2]: [cs_bill_customer_sk#10, cs_sold_date_sk#11] Condition : isnotnull(cs_bill_customer_sk#10) -(20) ReusedExchange [Reuses operator id: 6] +(19) ReusedExchange [Reuses operator id: 6] Output [2]: [d_date_sk#13, d_date#14] -(21) CometBroadcastHashJoin +(20) 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 -(22) CometProject +(21) 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] -(23) ReusedExchange [Reuses operator id: 11] +(22) ReusedExchange [Reuses operator id: 11] Output [3]: [c_customer_sk#15, c_first_name#16, c_last_name#17] -(24) CometBroadcastHashJoin +(23) 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 -(25) CometProject +(24) 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] -(26) CometHashAggregate +(25) 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: [] -(27) ColumnarToRow [codegen id : 2] +(26) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=2] -(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] +(27) 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#17, c_first_name#16, d_date#14] -(30) BroadcastExchange +(28) CometBroadcastExchange 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] +Arguments: [c_last_name#17, c_first_name#16, d_date#14] -(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 +(29) CometBroadcastHashJoin +Left output [3]: [c_last_name#9, c_first_name#8, d_date#5] +Right output [3]: [c_last_name#17, c_first_name#16, d_date#14] +Arguments: [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)], [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)], LeftSemi, BuildRight -(32) Scan parquet spark_catalog.default.web_sales +(30) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] @@ -206,119 +193,111 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#19), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(33) CometFilter +(31) CometFilter Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] Condition : isnotnull(ws_bill_customer_sk#18) -(34) ReusedExchange [Reuses operator id: 6] +(32) ReusedExchange [Reuses operator id: 6] Output [2]: [d_date_sk#21, d_date#22] -(35) CometBroadcastHashJoin +(33) 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 -(36) CometProject +(34) 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] -(37) ReusedExchange [Reuses operator id: 11] +(35) ReusedExchange [Reuses operator id: 11] Output [3]: [c_customer_sk#23, c_first_name#24, c_last_name#25] -(38) CometBroadcastHashJoin +(36) 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 -(39) CometProject +(37) 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] -(40) CometHashAggregate +(38) 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: [] -(41) ColumnarToRow [codegen id : 4] +(39) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=3] -(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] - -(43) HashAggregate [codegen id : 5] +(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#25, c_first_name#24, d_date#22] -(44) BroadcastExchange +(41) CometBroadcastExchange 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] +Arguments: [c_last_name#25, c_first_name#24, d_date#22] -(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 +(42) CometBroadcastHashJoin +Left output [3]: [c_last_name#9, c_first_name#8, d_date#5] +Right output [3]: [c_last_name#25, c_first_name#24, d_date#22] +Arguments: [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)], [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)], LeftSemi, BuildRight -(46) Project [codegen id : 6] -Output: [] +(43) CometProject Input [3]: [c_last_name#9, c_first_name#8, d_date#5] -(47) HashAggregate [codegen id : 6] +(44) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#26] -Results [1]: [count#27] -(48) Exchange -Input [1]: [count#27] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +(45) CometColumnarExchange +Input [1]: [count#26] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(49) HashAggregate [codegen id : 7] -Input [1]: [count#27] +(46) CometHashAggregate +Input [1]: [count#26] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#28] -Results [1]: [count(1)#28 AS count(1)#29] + +(47) ColumnarToRow [codegen id : 1] +Input [1]: [count(1)#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 -BroadcastExchange (54) -+- * ColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan parquet spark_catalog.default.date_dim (50) +BroadcastExchange (52) ++- * ColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) -(50) Scan parquet spark_catalog.default.date_dim +(48) 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 -(51) CometFilter +(49) 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)) -(52) CometProject +(50) 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] -(53) ColumnarToRow [codegen id : 1] +(51) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#4, d_date#5] -(54) BroadcastExchange +(52) BroadcastExchange Input [2]: [d_date_sk#4, d_date#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=5] -Subquery:2 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#11 IN dynamicpruning#3 +Subquery:2 Hosting operator id = 17 Hosting Expression = cs_sold_date_sk#11 IN dynamicpruning#3 -Subquery:3 Hosting operator id = 32 Hosting Expression = ws_sold_date_sk#19 IN dynamicpruning#3 +Subquery:3 Hosting operator id = 30 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 9d667265c..77970e8d3 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,75 +1,59 @@ -WholeStageCodegen (7) - HashAggregate [count] [count(1),count(1),count] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (6) - HashAggregate [count,count] - Project - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #2 - 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,d_date,c_customer_sk,c_first_name,c_last_name] - CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_customer_sk,ss_sold_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,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #4 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,d_date] - 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,d_date,c_customer_sk,c_first_name,c_last_name] - CometProject [cs_bill_customer_sk,d_date] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] - CometFilter [cs_bill_customer_sk,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_date] #4 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - HashAggregate [c_last_name,c_first_name,d_date] - 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,d_date,c_customer_sk,c_first_name,c_last_name] - CometProject [ws_bill_customer_sk,d_date] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_bill_customer_sk,ws_sold_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,d_date] #4 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 + CometHashAggregate [count(1),count,count(1)] + CometColumnarExchange #1 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #2 + CometHashAggregate [c_last_name,c_first_name,d_date] + CometProject [c_last_name,c_first_name,d_date] + CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] + CometProject [ss_customer_sk,d_date] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_customer_sk,ss_sold_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,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange [d_date_sk,d_date] #4 + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 + CometFilter [c_customer_sk,c_first_name,c_last_name] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] + CometBroadcastExchange [c_last_name,c_first_name,d_date] #6 + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #7 + CometHashAggregate [c_last_name,c_first_name,d_date] + CometProject [c_last_name,c_first_name,d_date] + CometBroadcastHashJoin [cs_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] + CometProject [cs_bill_customer_sk,d_date] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] + CometFilter [cs_bill_customer_sk,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_date] #4 + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 + CometBroadcastExchange [c_last_name,c_first_name,d_date] #8 + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #9 + CometHashAggregate [c_last_name,c_first_name,d_date] + CometProject [c_last_name,c_first_name,d_date] + CometBroadcastHashJoin [ws_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] + CometProject [ws_bill_customer_sk,d_date] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_bill_customer_sk,ws_sold_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,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 6b6272590..60c02c542 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,52 +1,51 @@ == Physical Plan == -* 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) +* ColumnarToRow (47) ++- CometSort (46) + +- CometColumnarExchange (45) + +- CometBroadcastHashJoin (44) + :- CometProject (23) + : +- CometFilter (22) + : +- CometHashAggregate (21) + : +- CometColumnarExchange (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) + +- CometBroadcastExchange (43) + +- CometProject (42) + +- CometFilter (41) + +- CometHashAggregate (40) + +- CometColumnarExchange (39) + +- CometHashAggregate (38) + +- CometProject (37) + +- CometBroadcastHashJoin (36) + :- CometProject (31) + : +- CometBroadcastHashJoin (30) + : :- CometProject (28) + : : +- CometBroadcastHashJoin (27) + : : :- CometFilter (25) + : : : +- CometScan parquet spark_catalog.default.inventory (24) + : : +- ReusedExchange (26) + : +- ReusedExchange (29) + +- CometBroadcastExchange (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan parquet spark_catalog.default.date_dim (32) (1) Scan parquet spark_catalog.default.inventory @@ -142,197 +141,189 @@ Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_n 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)] -(20) ColumnarToRow [codegen id : 1] +(20) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(22) HashAggregate [codegen id : 4] +(21) CometHashAggregate 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))#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] -(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 +(22) CometFilter +Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#17, mean#18] +Condition : CASE WHEN (mean#18 = 0.0) THEN false ELSE ((stdev#17 / mean#18) > 1.0) END -(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] +(23) CometProject +Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#17, mean#18] +Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19], [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, CASE WHEN (mean#18 = 0.0) THEN null ELSE (stdev#17 / mean#18) END AS cov#19] -(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] +(24) Scan parquet spark_catalog.default.inventory +Output [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#25), dynamicpruningexpression(inv_date_sk#25 IN dynamicpruning#26)] +PartitionFilters: [isnotnull(inv_date_sk#23), dynamicpruningexpression(inv_date_sk#23 IN dynamicpruning#24)] PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(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)) +(25) CometFilter +Input [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] +Condition : (isnotnull(inv_item_sk#20) AND isnotnull(inv_warehouse_sk#21)) -(27) ReusedExchange [Reuses operator id: 5] -Output [1]: [i_item_sk#27] +(26) ReusedExchange [Reuses operator id: 5] +Output [1]: [i_item_sk#25] -(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 +(27) CometBroadcastHashJoin +Left output [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] +Right output [1]: [i_item_sk#25] +Arguments: [inv_item_sk#20], [i_item_sk#25], Inner, BuildRight -(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] +(28) CometProject +Input [5]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] +Arguments: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25], [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] -(30) ReusedExchange [Reuses operator id: 10] -Output [2]: [w_warehouse_sk#28, w_warehouse_name#29] +(29) ReusedExchange [Reuses operator id: 10] +Output [2]: [w_warehouse_sk#26, w_warehouse_name#27] -(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 +(30) CometBroadcastHashJoin +Left output [4]: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] +Right output [2]: [w_warehouse_sk#26, w_warehouse_name#27] +Arguments: [inv_warehouse_sk#21], [w_warehouse_sk#26], Inner, BuildRight -(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] +(31) CometProject +Input [6]: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] +Arguments: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27], [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] -(33) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#30, d_year#31, d_moy#32] +(32) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#28, d_year#29, d_moy#30] 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 -(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)) +(33) CometFilter +Input [3]: [d_date_sk#28, d_year#29, d_moy#30] +Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2001)) AND (d_moy#30 = 2)) AND isnotnull(d_date_sk#28)) -(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] +(34) CometProject +Input [3]: [d_date_sk#28, d_year#29, d_moy#30] +Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] -(36) CometBroadcastExchange -Input [2]: [d_date_sk#30, d_moy#32] -Arguments: [d_date_sk#30, d_moy#32] +(35) CometBroadcastExchange +Input [2]: [d_date_sk#28, d_moy#30] +Arguments: [d_date_sk#28, d_moy#30] -(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 +(36) CometBroadcastHashJoin +Left output [5]: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] +Right output [2]: [d_date_sk#28, d_moy#30] +Arguments: [inv_date_sk#23], [d_date_sk#28], 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] +(37) CometProject +Input [7]: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_date_sk#28, d_moy#30] +Arguments: [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30], [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30] -(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)] +(38) CometHashAggregate +Input [5]: [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30] +Keys [4]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#22 as double)), partial_avg(inv_quantity_on_hand#22)] -(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] +(39) CometColumnarExchange +Input [9]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, n#31, avg#32, m2#33, sum#34, count#35] +Arguments: hashpartitioning(w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(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] +(40) CometHashAggregate +Input [9]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, n#31, avg#32, m2#33, sum#34, count#35] +Keys [4]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#22 as double)), avg(inv_quantity_on_hand#22)] -(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] +(41) CometFilter +Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, stdev#17, mean#18] +Condition : CASE WHEN (mean#18 = 0.0) THEN false ELSE ((stdev#17 / mean#18) > 1.0) END -(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 +(42) CometProject +Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, stdev#17, mean#18] +Arguments: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37], [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#18 AS mean#36, CASE WHEN (mean#18 = 0.0) THEN null ELSE (stdev#17 / mean#18) END AS cov#37] -(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] +(43) CometBroadcastExchange +Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] +Arguments: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -(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] +(44) CometBroadcastHashJoin +Left output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19] +Right output [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] +Arguments: [i_item_sk#6, w_warehouse_sk#7], [i_item_sk#25, w_warehouse_sk#26], Inner, BuildRight -(46) BroadcastHashJoin [codegen id : 4] -Left keys [2]: [i_item_sk#6, w_warehouse_sk#7] -Right keys [2]: [i_item_sk#27, w_warehouse_sk#28] -Join type: Inner -Join condition: None +(45) CometColumnarExchange +Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] +Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(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] +(46) CometSort +Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] +Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST] -(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 +(47) ColumnarToRow [codegen id : 1] +Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (53) -+- * ColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometScan parquet spark_catalog.default.date_dim (49) +BroadcastExchange (52) ++- * ColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) -(49) Scan parquet spark_catalog.default.date_dim +(48) 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 -(50) CometFilter +(49) 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)) -(51) CometProject +(50) 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] -(52) ColumnarToRow [codegen id : 1] +(51) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_moy#11] -(53) BroadcastExchange +(52) BroadcastExchange Input [2]: [d_date_sk#9, d_moy#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] -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) +Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 +BroadcastExchange (57) ++- * ColumnarToRow (56) + +- CometProject (55) + +- CometFilter (54) + +- CometScan parquet spark_catalog.default.date_dim (53) -(54) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#30, d_year#31, d_moy#32] +(53) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#28, d_year#29, d_moy#30] 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 -(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)) +(54) CometFilter +Input [3]: [d_date_sk#28, d_year#29, d_moy#30] +Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2001)) AND (d_moy#30 = 2)) AND isnotnull(d_date_sk#28)) -(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] +(55) CometProject +Input [3]: [d_date_sk#28, d_year#29, d_moy#30] +Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] -(57) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#30, d_moy#32] +(56) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#28, d_moy#30] -(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] +(57) BroadcastExchange +Input [2]: [d_date_sk#28, d_moy#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] 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 0c709e4f7..a4d1753f9 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,75 +1,65 @@ -WholeStageCodegen (5) - Sort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] - CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [i_item_sk] #4 - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [d_date_sk,d_moy] #6 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - 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] #8 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] - CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 #9 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - ReusedExchange [i_item_sk] #4 - ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - CometBroadcastExchange [d_date_sk,d_moy] #10 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] + CometColumnarExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 + CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] + CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] + CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] + CometHashAggregate [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,w_warehouse_name,n,avg,m2,sum,count,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] + CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] + CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] + CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [i_item_sk] #4 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange [d_date_sk,d_moy] #6 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov] #7 + CometProject [mean,stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] + CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] + CometHashAggregate [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,w_warehouse_name,n,avg,m2,sum,count,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] + CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] + CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] + CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 #9 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + ReusedExchange [i_item_sk] #4 + ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 + CometBroadcastExchange [d_date_sk,d_moy] #10 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + 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 85590bf81..d95f2c4cc 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,52 +1,51 @@ == Physical Plan == -* 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) +* ColumnarToRow (47) ++- CometSort (46) + +- CometColumnarExchange (45) + +- CometBroadcastHashJoin (44) + :- CometProject (23) + : +- CometFilter (22) + : +- CometHashAggregate (21) + : +- CometColumnarExchange (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) + +- CometBroadcastExchange (43) + +- CometProject (42) + +- CometFilter (41) + +- CometHashAggregate (40) + +- CometColumnarExchange (39) + +- CometHashAggregate (38) + +- CometProject (37) + +- CometBroadcastHashJoin (36) + :- CometProject (31) + : +- CometBroadcastHashJoin (30) + : :- CometProject (28) + : : +- CometBroadcastHashJoin (27) + : : :- CometFilter (25) + : : : +- CometScan parquet spark_catalog.default.inventory (24) + : : +- ReusedExchange (26) + : +- ReusedExchange (29) + +- CometBroadcastExchange (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan parquet spark_catalog.default.date_dim (32) (1) Scan parquet spark_catalog.default.inventory @@ -142,197 +141,189 @@ Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_n 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)] -(20) ColumnarToRow [codegen id : 1] +(20) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(22) HashAggregate [codegen id : 4] +(21) CometHashAggregate 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))#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] -(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) +(22) CometFilter +Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#17, mean#18] +Condition : (CASE WHEN (mean#18 = 0.0) THEN false ELSE ((stdev#17 / mean#18) > 1.0) END AND CASE WHEN (mean#18 = 0.0) THEN false ELSE ((stdev#17 / mean#18) > 1.5) END) -(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] +(23) CometProject +Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#17, mean#18] +Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19], [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, CASE WHEN (mean#18 = 0.0) THEN null ELSE (stdev#17 / mean#18) END AS cov#19] -(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] +(24) Scan parquet spark_catalog.default.inventory +Output [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#25), dynamicpruningexpression(inv_date_sk#25 IN dynamicpruning#26)] +PartitionFilters: [isnotnull(inv_date_sk#23), dynamicpruningexpression(inv_date_sk#23 IN dynamicpruning#24)] PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(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)) +(25) CometFilter +Input [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] +Condition : (isnotnull(inv_item_sk#20) AND isnotnull(inv_warehouse_sk#21)) -(27) ReusedExchange [Reuses operator id: 5] -Output [1]: [i_item_sk#27] +(26) ReusedExchange [Reuses operator id: 5] +Output [1]: [i_item_sk#25] -(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 +(27) CometBroadcastHashJoin +Left output [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] +Right output [1]: [i_item_sk#25] +Arguments: [inv_item_sk#20], [i_item_sk#25], Inner, BuildRight -(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] +(28) CometProject +Input [5]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] +Arguments: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25], [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] -(30) ReusedExchange [Reuses operator id: 10] -Output [2]: [w_warehouse_sk#28, w_warehouse_name#29] +(29) ReusedExchange [Reuses operator id: 10] +Output [2]: [w_warehouse_sk#26, w_warehouse_name#27] -(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 +(30) CometBroadcastHashJoin +Left output [4]: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] +Right output [2]: [w_warehouse_sk#26, w_warehouse_name#27] +Arguments: [inv_warehouse_sk#21], [w_warehouse_sk#26], Inner, BuildRight -(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] +(31) CometProject +Input [6]: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] +Arguments: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27], [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] -(33) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#30, d_year#31, d_moy#32] +(32) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#28, d_year#29, d_moy#30] 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 -(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)) +(33) CometFilter +Input [3]: [d_date_sk#28, d_year#29, d_moy#30] +Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2001)) AND (d_moy#30 = 2)) AND isnotnull(d_date_sk#28)) -(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] +(34) CometProject +Input [3]: [d_date_sk#28, d_year#29, d_moy#30] +Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] -(36) CometBroadcastExchange -Input [2]: [d_date_sk#30, d_moy#32] -Arguments: [d_date_sk#30, d_moy#32] +(35) CometBroadcastExchange +Input [2]: [d_date_sk#28, d_moy#30] +Arguments: [d_date_sk#28, d_moy#30] -(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 +(36) CometBroadcastHashJoin +Left output [5]: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] +Right output [2]: [d_date_sk#28, d_moy#30] +Arguments: [inv_date_sk#23], [d_date_sk#28], 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] +(37) CometProject +Input [7]: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_date_sk#28, d_moy#30] +Arguments: [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30], [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30] -(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)] +(38) CometHashAggregate +Input [5]: [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30] +Keys [4]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#22 as double)), partial_avg(inv_quantity_on_hand#22)] -(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] +(39) CometColumnarExchange +Input [9]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, n#31, avg#32, m2#33, sum#34, count#35] +Arguments: hashpartitioning(w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(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] +(40) CometHashAggregate +Input [9]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, n#31, avg#32, m2#33, sum#34, count#35] +Keys [4]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#22 as double)), avg(inv_quantity_on_hand#22)] -(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] +(41) CometFilter +Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, stdev#17, mean#18] +Condition : CASE WHEN (mean#18 = 0.0) THEN false ELSE ((stdev#17 / mean#18) > 1.0) END -(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 +(42) CometProject +Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, stdev#17, mean#18] +Arguments: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37], [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#18 AS mean#36, CASE WHEN (mean#18 = 0.0) THEN null ELSE (stdev#17 / mean#18) END AS cov#37] -(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] +(43) CometBroadcastExchange +Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] +Arguments: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -(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] +(44) CometBroadcastHashJoin +Left output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19] +Right output [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] +Arguments: [i_item_sk#6, w_warehouse_sk#7], [i_item_sk#25, w_warehouse_sk#26], Inner, BuildRight -(46) BroadcastHashJoin [codegen id : 4] -Left keys [2]: [i_item_sk#6, w_warehouse_sk#7] -Right keys [2]: [i_item_sk#27, w_warehouse_sk#28] -Join type: Inner -Join condition: None +(45) CometColumnarExchange +Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] +Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(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] +(46) CometSort +Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] +Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST] -(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 +(47) ColumnarToRow [codegen id : 1] +Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (53) -+- * ColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometScan parquet spark_catalog.default.date_dim (49) +BroadcastExchange (52) ++- * ColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) -(49) Scan parquet spark_catalog.default.date_dim +(48) 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 -(50) CometFilter +(49) 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)) -(51) CometProject +(50) 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] -(52) ColumnarToRow [codegen id : 1] +(51) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_moy#11] -(53) BroadcastExchange +(52) BroadcastExchange Input [2]: [d_date_sk#9, d_moy#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] -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) +Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 +BroadcastExchange (57) ++- * ColumnarToRow (56) + +- CometProject (55) + +- CometFilter (54) + +- CometScan parquet spark_catalog.default.date_dim (53) -(54) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#30, d_year#31, d_moy#32] +(53) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#28, d_year#29, d_moy#30] 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 -(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)) +(54) CometFilter +Input [3]: [d_date_sk#28, d_year#29, d_moy#30] +Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2001)) AND (d_moy#30 = 2)) AND isnotnull(d_date_sk#28)) -(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] +(55) CometProject +Input [3]: [d_date_sk#28, d_year#29, d_moy#30] +Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] -(57) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#30, d_moy#32] +(56) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#28, d_moy#30] -(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] +(57) BroadcastExchange +Input [2]: [d_date_sk#28, d_moy#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] 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 0c709e4f7..a4d1753f9 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,75 +1,65 @@ -WholeStageCodegen (5) - Sort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] - CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [i_item_sk] #4 - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [d_date_sk,d_moy] #6 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - 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] #8 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] - CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 #9 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - ReusedExchange [i_item_sk] #4 - ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - CometBroadcastExchange [d_date_sk,d_moy] #10 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] + CometColumnarExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 + CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] + CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] + CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] + CometHashAggregate [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,w_warehouse_name,n,avg,m2,sum,count,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] + CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] + CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] + CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [i_item_sk] #4 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange [d_date_sk,d_moy] #6 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov] #7 + CometProject [mean,stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] + CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] + CometHashAggregate [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,w_warehouse_name,n,avg,m2,sum,count,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] + CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] + CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] + CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 #9 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + ReusedExchange [i_item_sk] #4 + ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 + CometBroadcastExchange [d_date_sk,d_moy] #10 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + 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 4aea5922b..960cb130c 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,110 +1,105 @@ == Physical Plan == -TakeOrderedAndProject (106) -+- * Project (105) - +- * BroadcastHashJoin Inner BuildRight (104) - :- * Project (88) - : +- * BroadcastHashJoin Inner BuildRight (87) - : :- * Project (70) - : : +- * BroadcastHashJoin Inner BuildRight (69) - : : :- * 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) - : : +- * 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) +* ColumnarToRow (101) ++- CometTakeOrderedAndProject (100) + +- CometProject (99) + +- CometBroadcastHashJoin (98) + :- CometProject (83) + : +- CometBroadcastHashJoin (82) + : :- CometProject (66) + : : +- CometBroadcastHashJoin (65) + : : :- CometProject (50) + : : : +- CometBroadcastHashJoin (49) + : : : :- CometBroadcastHashJoin (33) + : : : : :- CometFilter (16) + : : : : : +- CometHashAggregate (15) + : : : : : +- CometColumnarExchange (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) + : : : : +- CometBroadcastExchange (32) + : : : : +- CometHashAggregate (31) + : : : : +- CometColumnarExchange (30) + : : : : +- CometHashAggregate (29) + : : : : +- CometProject (28) + : : : : +- CometBroadcastHashJoin (27) + : : : : :- CometProject (23) + : : : : : +- CometBroadcastHashJoin (22) + : : : : : :- CometFilter (18) + : : : : : : +- CometScan parquet spark_catalog.default.customer (17) + : : : : : +- CometBroadcastExchange (21) + : : : : : +- CometFilter (20) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (19) + : : : : +- CometBroadcastExchange (26) + : : : : +- CometFilter (25) + : : : : +- CometScan parquet spark_catalog.default.date_dim (24) + : : : +- CometBroadcastExchange (48) + : : : +- CometFilter (47) + : : : +- CometHashAggregate (46) + : : : +- CometColumnarExchange (45) + : : : +- CometHashAggregate (44) + : : : +- CometProject (43) + : : : +- CometBroadcastHashJoin (42) + : : : :- CometProject (40) + : : : : +- CometBroadcastHashJoin (39) + : : : : :- CometFilter (35) + : : : : : +- CometScan parquet spark_catalog.default.customer (34) + : : : : +- CometBroadcastExchange (38) + : : : : +- CometFilter (37) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (36) + : : : +- ReusedExchange (41) + : : +- CometBroadcastExchange (64) + : : +- CometHashAggregate (63) + : : +- CometColumnarExchange (62) + : : +- CometHashAggregate (61) + : : +- CometProject (60) + : : +- CometBroadcastHashJoin (59) + : : :- CometProject (57) + : : : +- CometBroadcastHashJoin (56) + : : : :- CometFilter (52) + : : : : +- CometScan parquet spark_catalog.default.customer (51) + : : : +- CometBroadcastExchange (55) + : : : +- CometFilter (54) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (53) + : : +- ReusedExchange (58) + : +- CometBroadcastExchange (81) + : +- CometFilter (80) + : +- CometHashAggregate (79) + : +- CometColumnarExchange (78) + : +- CometHashAggregate (77) + : +- CometProject (76) + : +- CometBroadcastHashJoin (75) + : :- CometProject (73) + : : +- CometBroadcastHashJoin (72) + : : :- CometFilter (68) + : : : +- CometScan parquet spark_catalog.default.customer (67) + : : +- CometBroadcastExchange (71) + : : +- CometFilter (70) + : : +- CometScan parquet spark_catalog.default.web_sales (69) + : +- ReusedExchange (74) + +- CometBroadcastExchange (97) + +- CometHashAggregate (96) + +- CometColumnarExchange (95) + +- CometHashAggregate (94) + +- CometProject (93) + +- CometBroadcastHashJoin (92) + :- CometProject (90) + : +- CometBroadcastHashJoin (89) + : :- CometFilter (85) + : : +- CometScan parquet spark_catalog.default.customer (84) + : +- CometBroadcastExchange (88) + : +- CometFilter (87) + : +- CometScan parquet spark_catalog.default.web_sales (86) + +- ReusedExchange (91) (1) Scan parquet spark_catalog.default.customer @@ -172,507 +167,475 @@ Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_fl 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))] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(15) 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, 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))#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] -(17) Filter [codegen id : 12] -Input [2]: [customer_id#21, year_total#22] -Condition : (isnotnull(year_total#22) AND (year_total#22 > 0.000000)) +(16) CometFilter +Input [2]: [customer_id#20, year_total#21] +Condition : (isnotnull(year_total#21) AND (year_total#21 > 0.000000)) -(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] +(17) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#22, c_customer_id#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_country#27, c_login#28, c_email_address#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(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)) +(18) CometFilter +Input [8]: [c_customer_sk#22, c_customer_id#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_country#27, c_login#28, c_email_address#29] +Condition : (isnotnull(c_customer_sk#22) AND isnotnull(c_customer_id#23)) -(20) Scan parquet spark_catalog.default.store_sales -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] +(19) Scan parquet spark_catalog.default.store_sales +Output [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#36), dynamicpruningexpression(ss_sold_date_sk#36 IN dynamicpruning#37)] +PartitionFilters: [isnotnull(ss_sold_date_sk#35), dynamicpruningexpression(ss_sold_date_sk#35 IN dynamicpruning#36)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(21) CometFilter -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) +(20) CometFilter +Input [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Condition : isnotnull(ss_customer_sk#30) -(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] +(21) CometBroadcastExchange +Input [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Arguments: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35] -(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 +(22) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#22, c_customer_id#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_country#27, c_login#28, c_email_address#29] +Right output [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Arguments: [c_customer_sk#22], [ss_customer_sk#30], Inner, BuildRight -(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] +(23) CometProject +Input [14]: [c_customer_sk#22, c_customer_id#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_country#27, c_login#28, c_email_address#29, ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Arguments: [c_customer_id#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_country#27, c_login#28, c_email_address#29, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35], [c_customer_id#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_country#27, c_login#28, c_email_address#29, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35] -(25) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#38, d_year#39] +(24) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#37, d_year#38] 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 -(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 - -(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] +(25) CometFilter +Input [2]: [d_date_sk#37, d_year#38] +Condition : ((isnotnull(d_year#38) AND (d_year#38 = 2002)) AND isnotnull(d_date_sk#37)) + +(26) CometBroadcastExchange +Input [2]: [d_date_sk#37, d_year#38] +Arguments: [d_date_sk#37, d_year#38] + +(27) CometBroadcastHashJoin +Left output [12]: [c_customer_id#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_country#27, c_login#28, c_email_address#29, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Right output [2]: [d_date_sk#37, d_year#38] +Arguments: [ss_sold_date_sk#35], [d_date_sk#37], Inner, BuildRight + +(28) CometProject +Input [14]: [c_customer_id#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_country#27, c_login#28, c_email_address#29, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35, d_date_sk#37, d_year#38] +Arguments: [c_customer_id#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_country#27, c_login#28, c_email_address#29, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38], [c_customer_id#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_country#27, c_login#28, c_email_address#29, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38] + +(29) CometHashAggregate +Input [12]: [c_customer_id#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_country#27, c_login#28, c_email_address#29, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38] +Keys [8]: [c_customer_id#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_country#27, c_login#28, c_email_address#29, d_year#38] +Functions [1]: [partial_sum(((((ss_ext_list_price#34 - ss_ext_wholesale_cost#33) - ss_ext_discount_amt#31) + ss_ext_sales_price#32) / 2))] + +(30) CometColumnarExchange +Input [10]: [c_customer_id#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_country#27, c_login#28, c_email_address#29, d_year#38, sum#39, isEmpty#40] +Arguments: hashpartitioning(c_customer_id#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_country#27, c_login#28, c_email_address#29, d_year#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(31) CometHashAggregate +Input [10]: [c_customer_id#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_country#27, c_login#28, c_email_address#29, d_year#38, sum#39, isEmpty#40] +Keys [8]: [c_customer_id#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26, c_birth_country#27, c_login#28, c_email_address#29, d_year#38] +Functions [1]: [sum(((((ss_ext_list_price#34 - ss_ext_wholesale_cost#33) - ss_ext_discount_amt#31) + ss_ext_sales_price#32) / 2))] + +(32) CometBroadcastExchange +Input [8]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47, year_total#48] +Arguments: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47, year_total#48] + +(33) CometBroadcastHashJoin +Left output [2]: [customer_id#20, year_total#21] +Right output [8]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47, year_total#48] +Arguments: [customer_id#20], [customer_id#41], Inner, BuildRight + +(34) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#49, c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(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)) +(35) CometFilter +Input [8]: [c_customer_sk#49, c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56] +Condition : (isnotnull(c_customer_sk#49) AND isnotnull(c_customer_id#50)) -(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] +(36) Scan parquet spark_catalog.default.catalog_sales +Output [6]: [cs_bill_customer_sk#57, cs_ext_discount_amt#58, cs_ext_sales_price#59, cs_ext_wholesale_cost#60, cs_ext_list_price#61, cs_sold_date_sk#62] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#63), dynamicpruningexpression(cs_sold_date_sk#63 IN dynamicpruning#64)] +PartitionFilters: [isnotnull(cs_sold_date_sk#62), dynamicpruningexpression(cs_sold_date_sk#62 IN dynamicpruning#63)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(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) +(37) CometFilter +Input [6]: [cs_bill_customer_sk#57, cs_ext_discount_amt#58, cs_ext_sales_price#59, cs_ext_wholesale_cost#60, cs_ext_list_price#61, cs_sold_date_sk#62] +Condition : isnotnull(cs_bill_customer_sk#57) + +(38) CometBroadcastExchange +Input [6]: [cs_bill_customer_sk#57, cs_ext_discount_amt#58, cs_ext_sales_price#59, cs_ext_wholesale_cost#60, cs_ext_list_price#61, cs_sold_date_sk#62] +Arguments: [cs_bill_customer_sk#57, cs_ext_discount_amt#58, cs_ext_sales_price#59, cs_ext_wholesale_cost#60, cs_ext_list_price#61, cs_sold_date_sk#62] + +(39) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#49, c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56] +Right output [6]: [cs_bill_customer_sk#57, cs_ext_discount_amt#58, cs_ext_sales_price#59, cs_ext_wholesale_cost#60, cs_ext_list_price#61, cs_sold_date_sk#62] +Arguments: [c_customer_sk#49], [cs_bill_customer_sk#57], Inner, BuildRight -(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) CometProject +Input [14]: [c_customer_sk#49, c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, cs_bill_customer_sk#57, cs_ext_discount_amt#58, cs_ext_sales_price#59, cs_ext_wholesale_cost#60, cs_ext_list_price#61, cs_sold_date_sk#62] +Arguments: [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, cs_ext_discount_amt#58, cs_ext_sales_price#59, cs_ext_wholesale_cost#60, cs_ext_list_price#61, cs_sold_date_sk#62], [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, cs_ext_discount_amt#58, cs_ext_sales_price#59, cs_ext_wholesale_cost#60, cs_ext_list_price#61, cs_sold_date_sk#62] -(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) ReusedExchange [Reuses operator id: 10] +Output [2]: [d_date_sk#64, d_year#65] -(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) CometBroadcastHashJoin +Left output [12]: [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, cs_ext_discount_amt#58, cs_ext_sales_price#59, cs_ext_wholesale_cost#60, cs_ext_list_price#61, cs_sold_date_sk#62] +Right output [2]: [d_date_sk#64, d_year#65] +Arguments: [cs_sold_date_sk#62], [d_date_sk#64], Inner, BuildRight -(43) ReusedExchange [Reuses operator id: 10] -Output [2]: [d_date_sk#65, d_year#66] +(43) CometProject +Input [14]: [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, cs_ext_discount_amt#58, cs_ext_sales_price#59, cs_ext_wholesale_cost#60, cs_ext_list_price#61, cs_sold_date_sk#62, d_date_sk#64, d_year#65] +Arguments: [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, cs_ext_discount_amt#58, cs_ext_sales_price#59, cs_ext_wholesale_cost#60, cs_ext_list_price#61, d_year#65], [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, cs_ext_discount_amt#58, cs_ext_sales_price#59, cs_ext_wholesale_cost#60, cs_ext_list_price#61, d_year#65] -(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) CometHashAggregate +Input [12]: [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, cs_ext_discount_amt#58, cs_ext_sales_price#59, cs_ext_wholesale_cost#60, cs_ext_list_price#61, d_year#65] +Keys [8]: [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, d_year#65] +Functions [1]: [partial_sum(((((cs_ext_list_price#61 - cs_ext_wholesale_cost#60) - cs_ext_discount_amt#58) + cs_ext_sales_price#59) / 2))] -(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] +(45) CometColumnarExchange +Input [10]: [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, d_year#65, sum#66, isEmpty#67] +Arguments: hashpartitioning(c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, d_year#65, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (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))] - -(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 - -(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] - -(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] +Input [10]: [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, d_year#65, sum#66, isEmpty#67] +Keys [8]: [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, d_year#65] +Functions [1]: [sum(((((cs_ext_list_price#61 - cs_ext_wholesale_cost#60) - cs_ext_discount_amt#58) + cs_ext_sales_price#59) / 2))] + +(47) CometFilter +Input [2]: [customer_id#68, year_total#69] +Condition : (isnotnull(year_total#69) AND (year_total#69 > 0.000000)) + +(48) CometBroadcastExchange +Input [2]: [customer_id#68, year_total#69] +Arguments: [customer_id#68, year_total#69] + +(49) CometBroadcastHashJoin +Left output [10]: [customer_id#20, year_total#21, customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47, year_total#48] +Right output [2]: [customer_id#68, year_total#69] +Arguments: [customer_id#20], [customer_id#68], Inner, BuildRight + +(50) CometProject +Input [12]: [customer_id#20, year_total#21, customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47, year_total#48, customer_id#68, year_total#69] +Arguments: [customer_id#20, year_total#21, customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47, year_total#48, year_total#69], [customer_id#20, year_total#21, customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47, year_total#48, year_total#69] + +(51) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] 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#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)) +(52) CometFilter +Input [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] +Condition : (isnotnull(c_customer_sk#70) AND isnotnull(c_customer_id#71)) -(56) Scan parquet spark_catalog.default.catalog_sales -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] +(53) Scan parquet spark_catalog.default.catalog_sales +Output [6]: [cs_bill_customer_sk#78, cs_ext_discount_amt#79, cs_ext_sales_price#80, cs_ext_wholesale_cost#81, cs_ext_list_price#82, cs_sold_date_sk#83] 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#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#84)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(57) CometFilter -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) +(54) CometFilter +Input [6]: [cs_bill_customer_sk#78, cs_ext_discount_amt#79, cs_ext_sales_price#80, cs_ext_wholesale_cost#81, cs_ext_list_price#82, cs_sold_date_sk#83] +Condition : isnotnull(cs_bill_customer_sk#78) + +(55) CometBroadcastExchange +Input [6]: [cs_bill_customer_sk#78, cs_ext_discount_amt#79, cs_ext_sales_price#80, cs_ext_wholesale_cost#81, cs_ext_list_price#82, cs_sold_date_sk#83] +Arguments: [cs_bill_customer_sk#78, cs_ext_discount_amt#79, cs_ext_sales_price#80, cs_ext_wholesale_cost#81, cs_ext_list_price#82, cs_sold_date_sk#83] + +(56) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] +Right output [6]: [cs_bill_customer_sk#78, cs_ext_discount_amt#79, cs_ext_sales_price#80, cs_ext_wholesale_cost#81, cs_ext_list_price#82, cs_sold_date_sk#83] +Arguments: [c_customer_sk#70], [cs_bill_customer_sk#78], Inner, BuildRight -(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] +(57) CometProject +Input [14]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, cs_bill_customer_sk#78, cs_ext_discount_amt#79, cs_ext_sales_price#80, cs_ext_wholesale_cost#81, cs_ext_list_price#82, cs_sold_date_sk#83] +Arguments: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, cs_ext_discount_amt#79, cs_ext_sales_price#80, cs_ext_wholesale_cost#81, cs_ext_list_price#82, cs_sold_date_sk#83], [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, cs_ext_discount_amt#79, cs_ext_sales_price#80, cs_ext_wholesale_cost#81, cs_ext_list_price#82, cs_sold_date_sk#83] + +(58) ReusedExchange [Reuses operator id: 26] +Output [2]: [d_date_sk#85, d_year#86] (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 +Left output [12]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, cs_ext_discount_amt#79, cs_ext_sales_price#80, cs_ext_wholesale_cost#81, cs_ext_list_price#82, cs_sold_date_sk#83] +Right output [2]: [d_date_sk#85, d_year#86] +Arguments: [cs_sold_date_sk#83], [d_date_sk#85], Inner, BuildRight (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) ReusedExchange [Reuses operator id: 27] -Output [2]: [d_date_sk#87, d_year#88] - -(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) 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) 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) 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#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 : 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#91, year_total#92] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] - -(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#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 : 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#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] +Input [14]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, cs_ext_discount_amt#79, cs_ext_sales_price#80, cs_ext_wholesale_cost#81, cs_ext_list_price#82, cs_sold_date_sk#83, d_date_sk#85, d_year#86] +Arguments: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, cs_ext_discount_amt#79, cs_ext_sales_price#80, cs_ext_wholesale_cost#81, cs_ext_list_price#82, d_year#86], [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, cs_ext_discount_amt#79, cs_ext_sales_price#80, cs_ext_wholesale_cost#81, cs_ext_list_price#82, d_year#86] + +(61) CometHashAggregate +Input [12]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, cs_ext_discount_amt#79, cs_ext_sales_price#80, cs_ext_wholesale_cost#81, cs_ext_list_price#82, d_year#86] +Keys [8]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, d_year#86] +Functions [1]: [partial_sum(((((cs_ext_list_price#82 - cs_ext_wholesale_cost#81) - cs_ext_discount_amt#79) + cs_ext_sales_price#80) / 2))] + +(62) CometColumnarExchange +Input [10]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, d_year#86, sum#87, isEmpty#88] +Arguments: hashpartitioning(c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, d_year#86, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(63) CometHashAggregate +Input [10]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, d_year#86, sum#87, isEmpty#88] +Keys [8]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, d_year#86] +Functions [1]: [sum(((((cs_ext_list_price#82 - cs_ext_wholesale_cost#81) - cs_ext_discount_amt#79) + cs_ext_sales_price#80) / 2))] + +(64) CometBroadcastExchange +Input [2]: [customer_id#89, year_total#90] +Arguments: [customer_id#89, year_total#90] + +(65) CometBroadcastHashJoin +Left output [11]: [customer_id#20, year_total#21, customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47, year_total#48, year_total#69] +Right output [2]: [customer_id#89, year_total#90] +Arguments: [customer_id#20], [customer_id#89], Inner, (CASE WHEN (year_total#69 > 0.000000) THEN (year_total#90 / year_total#69) END > CASE WHEN (year_total#21 > 0.000000) THEN (year_total#48 / year_total#21) END), BuildRight + +(66) CometProject +Input [13]: [customer_id#20, year_total#21, customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47, year_total#48, year_total#69, customer_id#89, year_total#90] +Arguments: [customer_id#20, customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47, year_total#69, year_total#90], [customer_id#20, customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47, year_total#69, year_total#90] + +(67) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#91, c_customer_id#92, c_first_name#93, c_last_name#94, c_preferred_cust_flag#95, c_birth_country#96, c_login#97, c_email_address#98] 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#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)) +(68) CometFilter +Input [8]: [c_customer_sk#91, c_customer_id#92, c_first_name#93, c_last_name#94, c_preferred_cust_flag#95, c_birth_country#96, c_login#97, c_email_address#98] +Condition : (isnotnull(c_customer_sk#91) AND isnotnull(c_customer_id#92)) -(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] +(69) Scan parquet spark_catalog.default.web_sales +Output [6]: [ws_bill_customer_sk#99, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, ws_sold_date_sk#104] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#106), dynamicpruningexpression(ws_sold_date_sk#106 IN dynamicpruning#107)] +PartitionFilters: [isnotnull(ws_sold_date_sk#104), dynamicpruningexpression(ws_sold_date_sk#104 IN dynamicpruning#105)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(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 - -(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] - -(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] +(70) CometFilter +Input [6]: [ws_bill_customer_sk#99, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, ws_sold_date_sk#104] +Condition : isnotnull(ws_bill_customer_sk#99) + +(71) CometBroadcastExchange +Input [6]: [ws_bill_customer_sk#99, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, ws_sold_date_sk#104] +Arguments: [ws_bill_customer_sk#99, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, ws_sold_date_sk#104] + +(72) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#91, c_customer_id#92, c_first_name#93, c_last_name#94, c_preferred_cust_flag#95, c_birth_country#96, c_login#97, c_email_address#98] +Right output [6]: [ws_bill_customer_sk#99, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, ws_sold_date_sk#104] +Arguments: [c_customer_sk#91], [ws_bill_customer_sk#99], Inner, BuildRight + +(73) CometProject +Input [14]: [c_customer_sk#91, c_customer_id#92, c_first_name#93, c_last_name#94, c_preferred_cust_flag#95, c_birth_country#96, c_login#97, c_email_address#98, ws_bill_customer_sk#99, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, ws_sold_date_sk#104] +Arguments: [c_customer_id#92, c_first_name#93, c_last_name#94, c_preferred_cust_flag#95, c_birth_country#96, c_login#97, c_email_address#98, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, ws_sold_date_sk#104], [c_customer_id#92, c_first_name#93, c_last_name#94, c_preferred_cust_flag#95, c_birth_country#96, c_login#97, c_email_address#98, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, ws_sold_date_sk#104] + +(74) ReusedExchange [Reuses operator id: 10] +Output [2]: [d_date_sk#106, d_year#107] + +(75) CometBroadcastHashJoin +Left output [12]: [c_customer_id#92, c_first_name#93, c_last_name#94, c_preferred_cust_flag#95, c_birth_country#96, c_login#97, c_email_address#98, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, ws_sold_date_sk#104] +Right output [2]: [d_date_sk#106, d_year#107] +Arguments: [ws_sold_date_sk#104], [d_date_sk#106], Inner, BuildRight + +(76) CometProject +Input [14]: [c_customer_id#92, c_first_name#93, c_last_name#94, c_preferred_cust_flag#95, c_birth_country#96, c_login#97, c_email_address#98, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, ws_sold_date_sk#104, d_date_sk#106, d_year#107] +Arguments: [c_customer_id#92, c_first_name#93, c_last_name#94, c_preferred_cust_flag#95, c_birth_country#96, c_login#97, c_email_address#98, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, d_year#107], [c_customer_id#92, c_first_name#93, c_last_name#94, c_preferred_cust_flag#95, c_birth_country#96, c_login#97, c_email_address#98, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, d_year#107] + +(77) CometHashAggregate +Input [12]: [c_customer_id#92, c_first_name#93, c_last_name#94, c_preferred_cust_flag#95, c_birth_country#96, c_login#97, c_email_address#98, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, d_year#107] +Keys [8]: [c_customer_id#92, c_first_name#93, c_last_name#94, c_preferred_cust_flag#95, c_birth_country#96, c_login#97, c_email_address#98, d_year#107] +Functions [1]: [partial_sum(((((ws_ext_list_price#103 - ws_ext_wholesale_cost#102) - ws_ext_discount_amt#100) + ws_ext_sales_price#101) / 2))] + +(78) CometColumnarExchange +Input [10]: [c_customer_id#92, c_first_name#93, c_last_name#94, c_preferred_cust_flag#95, c_birth_country#96, c_login#97, c_email_address#98, d_year#107, sum#108, isEmpty#109] +Arguments: hashpartitioning(c_customer_id#92, c_first_name#93, c_last_name#94, c_preferred_cust_flag#95, c_birth_country#96, c_login#97, c_email_address#98, d_year#107, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(79) CometHashAggregate +Input [10]: [c_customer_id#92, c_first_name#93, c_last_name#94, c_preferred_cust_flag#95, c_birth_country#96, c_login#97, c_email_address#98, d_year#107, sum#108, isEmpty#109] +Keys [8]: [c_customer_id#92, c_first_name#93, c_last_name#94, c_preferred_cust_flag#95, c_birth_country#96, c_login#97, c_email_address#98, d_year#107] +Functions [1]: [sum(((((ws_ext_list_price#103 - ws_ext_wholesale_cost#102) - ws_ext_discount_amt#100) + ws_ext_sales_price#101) / 2))] + +(80) CometFilter +Input [2]: [customer_id#110, year_total#111] +Condition : (isnotnull(year_total#111) AND (year_total#111 > 0.000000)) + +(81) CometBroadcastExchange +Input [2]: [customer_id#110, year_total#111] +Arguments: [customer_id#110, year_total#111] + +(82) CometBroadcastHashJoin +Left output [10]: [customer_id#20, customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47, year_total#69, year_total#90] +Right output [2]: [customer_id#110, year_total#111] +Arguments: [customer_id#20], [customer_id#110], Inner, BuildRight + +(83) CometProject +Input [12]: [customer_id#20, customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47, year_total#69, year_total#90, customer_id#110, year_total#111] +Arguments: [customer_id#20, customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47, year_total#69, year_total#90, year_total#111], [customer_id#20, customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47, year_total#69, year_total#90, year_total#111] + +(84) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#112, c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(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)) +(85) CometFilter +Input [8]: [c_customer_sk#112, c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119] +Condition : (isnotnull(c_customer_sk#112) AND isnotnull(c_customer_id#113)) -(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] +(86) Scan parquet spark_catalog.default.web_sales +Output [6]: [ws_bill_customer_sk#120, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, ws_sold_date_sk#125] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#128), dynamicpruningexpression(ws_sold_date_sk#128 IN dynamicpruning#129)] +PartitionFilters: [isnotnull(ws_sold_date_sk#125), dynamicpruningexpression(ws_sold_date_sk#125 IN dynamicpruning#126)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(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) +(87) CometFilter +Input [6]: [ws_bill_customer_sk#120, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, ws_sold_date_sk#125] +Condition : isnotnull(ws_bill_customer_sk#120) -(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] +(88) CometBroadcastExchange +Input [6]: [ws_bill_customer_sk#120, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, ws_sold_date_sk#125] +Arguments: [ws_bill_customer_sk#120, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, ws_sold_date_sk#125] -(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 +(89) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#112, c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119] +Right output [6]: [ws_bill_customer_sk#120, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, ws_sold_date_sk#125] +Arguments: [c_customer_sk#112], [ws_bill_customer_sk#120], Inner, BuildRight -(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] +(90) CometProject +Input [14]: [c_customer_sk#112, c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, ws_bill_customer_sk#120, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, ws_sold_date_sk#125] +Arguments: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, ws_sold_date_sk#125], [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, ws_sold_date_sk#125] -(96) ReusedExchange [Reuses operator id: 27] -Output [2]: [d_date_sk#130, d_year#131] +(91) ReusedExchange [Reuses operator id: 26] +Output [2]: [d_date_sk#127, d_year#128] -(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 +(92) CometBroadcastHashJoin +Left output [12]: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, ws_sold_date_sk#125] +Right output [2]: [d_date_sk#127, d_year#128] +Arguments: [ws_sold_date_sk#125], [d_date_sk#127], Inner, BuildRight -(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] +(93) CometProject +Input [14]: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, ws_sold_date_sk#125, d_date_sk#127, d_year#128] +Arguments: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, d_year#128], [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, d_year#128] -(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))] +(94) CometHashAggregate +Input [12]: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, d_year#128] +Keys [8]: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, d_year#128] +Functions [1]: [partial_sum(((((ws_ext_list_price#124 - ws_ext_wholesale_cost#123) - ws_ext_discount_amt#121) + ws_ext_sales_price#122) / 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] +(95) CometColumnarExchange +Input [10]: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, d_year#128, sum#129, isEmpty#130] +Arguments: hashpartitioning(c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, d_year#128, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(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] +(96) CometHashAggregate +Input [10]: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, d_year#128, sum#129, isEmpty#130] +Keys [8]: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, d_year#128] +Functions [1]: [sum(((((ws_ext_list_price#124 - ws_ext_wholesale_cost#123) - ws_ext_discount_amt#121) + ws_ext_sales_price#122) / 2))] -(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] +(97) CometBroadcastExchange +Input [2]: [customer_id#131, year_total#132] +Arguments: [customer_id#131, year_total#132] -(103) BroadcastExchange -Input [2]: [customer_id#134, year_total#135] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] +(98) CometBroadcastHashJoin +Left output [11]: [customer_id#20, customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47, year_total#69, year_total#90, year_total#111] +Right output [2]: [customer_id#131, year_total#132] +Arguments: [customer_id#20], [customer_id#131], Inner, (CASE WHEN (year_total#69 > 0.000000) THEN (year_total#90 / year_total#69) END > CASE WHEN (year_total#111 > 0.000000) THEN (year_total#132 / year_total#111) END), BuildRight -(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#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) +(99) CometProject +Input [13]: [customer_id#20, customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47, year_total#69, year_total#90, year_total#111, customer_id#131, year_total#132] +Arguments: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47], [customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47] -(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] +(100) CometTakeOrderedAndProject +Input [7]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#41 ASC NULLS FIRST,customer_first_name#42 ASC NULLS FIRST,customer_last_name#43 ASC NULLS FIRST,customer_preferred_cust_flag#44 ASC NULLS FIRST,customer_birth_country#45 ASC NULLS FIRST,customer_login#46 ASC NULLS FIRST,customer_email_address#47 ASC NULLS FIRST], output=[customer_id#41,customer_first_name#42,customer_last_name#43,customer_preferred_cust_flag#44,customer_birth_country#45,customer_login#46,customer_email_address#47]), [customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47], 100, [customer_id#41 ASC NULLS FIRST, customer_first_name#42 ASC NULLS FIRST, customer_last_name#43 ASC NULLS FIRST, customer_preferred_cust_flag#44 ASC NULLS FIRST, customer_birth_country#45 ASC NULLS FIRST, customer_login#46 ASC NULLS FIRST, customer_email_address#47 ASC NULLS FIRST], [customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47] -(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] +(101) ColumnarToRow [codegen id : 1] +Input [7]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_preferred_cust_flag#44, customer_birth_country#45, customer_login#46, customer_email_address#47] ===== Subqueries ===== 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) +BroadcastExchange (105) ++- * ColumnarToRow (104) + +- CometFilter (103) + +- CometScan parquet spark_catalog.default.date_dim (102) -(107) Scan parquet spark_catalog.default.date_dim +(102) 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 -(108) CometFilter +(103) 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)) -(109) ColumnarToRow [codegen id : 1] +(104) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#16, d_year#17] -(110) BroadcastExchange +(105) BroadcastExchange Input [2]: [d_date_sk#16, d_year#17] -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=7] -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) +Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#35 IN dynamicpruning#36 +BroadcastExchange (109) ++- * ColumnarToRow (108) + +- CometFilter (107) + +- CometScan parquet spark_catalog.default.date_dim (106) -(111) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#38, d_year#39] +(106) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#37, d_year#38] 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 -(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)) +(107) CometFilter +Input [2]: [d_date_sk#37, d_year#38] +Condition : ((isnotnull(d_year#38) AND (d_year#38 = 2002)) AND isnotnull(d_date_sk#37)) -(113) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#38, d_year#39] +(108) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#37, d_year#38] -(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] +(109) BroadcastExchange +Input [2]: [d_date_sk#37, d_year#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -Subquery:3 Hosting operator id = 38 Hosting Expression = cs_sold_date_sk#63 IN dynamicpruning#15 +Subquery:3 Hosting operator id = 36 Hosting Expression = cs_sold_date_sk#62 IN dynamicpruning#15 -Subquery:4 Hosting operator id = 56 Hosting Expression = cs_sold_date_sk#85 IN dynamicpruning#37 +Subquery:4 Hosting operator id = 53 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#36 -Subquery:5 Hosting operator id = 73 Hosting Expression = ws_sold_date_sk#106 IN dynamicpruning#15 +Subquery:5 Hosting operator id = 69 Hosting Expression = ws_sold_date_sk#104 IN dynamicpruning#15 -Subquery:6 Hosting operator id = 91 Hosting Expression = ws_sold_date_sk#128 IN dynamicpruning#37 +Subquery:6 Hosting operator id = 86 Hosting Expression = ws_sold_date_sk#125 IN dynamicpruning#36 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 1a19c58ff..7e6b12575 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,153 +1,121 @@ -TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] - 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] - BroadcastHashJoin [customer_id,customer_id] - 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] - 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_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id] - BroadcastHashJoin [customer_id,customer_id] - 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(((((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 (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,sum,isEmpty,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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_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_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - 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,sum,isEmpty,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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #7 - CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_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 #8 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - 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] #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,sum,isEmpty,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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #12 - CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_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 #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] #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,sum,isEmpty,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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #15 - CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_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 #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] #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,sum,isEmpty,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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] + CometProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] + CometBroadcastHashJoin [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,customer_id,year_total] + CometProject [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] + CometBroadcastHashJoin [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,customer_id,year_total] + CometProject [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] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,customer_id,year_total] + CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,customer_id,year_total] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total] + CometFilter [customer_id,year_total] + CometHashAggregate [customer_id,year_total,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))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #1 + CometHashAggregate [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,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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #18 - CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_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] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_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_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total] #5 + CometHashAggregate [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,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))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #6 + CometHashAggregate [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,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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + 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 [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_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 #8 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [customer_id,year_total] #10 + CometFilter [customer_id,year_total] + CometHashAggregate [customer_id,year_total,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))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 + CometHashAggregate [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,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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + 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 [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #12 + CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_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 #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] #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,sum,isEmpty,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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + CometBroadcastExchange [customer_id,year_total] #13 + CometHashAggregate [customer_id,year_total,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))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 + CometHashAggregate [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,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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #21 - CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_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] + CometBroadcastExchange [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #15 + CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_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 + CometBroadcastExchange [customer_id,year_total] #16 + CometFilter [customer_id,year_total] + CometHashAggregate [customer_id,year_total,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))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #17 + CometHashAggregate [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,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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #18 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_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 + CometBroadcastExchange [customer_id,year_total] #19 + CometHashAggregate [customer_id,year_total,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))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #20 + CometHashAggregate [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,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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #21 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_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/q40/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt index f63b94658..00967faa4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt @@ -1,37 +1,36 @@ == Physical Plan == -TakeOrderedAndProject (33) -+- * HashAggregate (32) - +- Exchange (31) - +- * HashAggregate (30) - +- * Project (29) - +- * BroadcastHashJoin Inner BuildRight (28) - :- * Project (26) - : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (13) - : : : +- * SortMergeJoin LeftOuter (12) - : : : :- * Sort (5) - : : : : +- Exchange (4) - : : : : +- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : +- * Sort (11) - : : : +- Exchange (10) - : : : +- * ColumnarToRow (9) - : : : +- CometProject (8) - : : : +- CometFilter (7) - : : : +- CometScan parquet spark_catalog.default.catalog_returns (6) - : : +- BroadcastExchange (17) - : : +- * ColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometScan parquet spark_catalog.default.warehouse (14) - : +- BroadcastExchange (24) - : +- * ColumnarToRow (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan parquet spark_catalog.default.item (20) - +- ReusedExchange (27) +* ColumnarToRow (32) ++- CometTakeOrderedAndProject (31) + +- CometHashAggregate (30) + +- CometColumnarExchange (29) + +- CometHashAggregate (28) + +- CometProject (27) + +- CometBroadcastHashJoin (26) + :- CometProject (22) + : +- CometBroadcastHashJoin (21) + : :- CometProject (16) + : : +- CometBroadcastHashJoin (15) + : : :- CometProject (11) + : : : +- CometSortMergeJoin (10) + : : : :- CometSort (4) + : : : : +- CometColumnarExchange (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : +- CometSort (9) + : : : +- CometColumnarExchange (8) + : : : +- CometProject (7) + : : : +- CometFilter (6) + : : : +- CometScan parquet spark_catalog.default.catalog_returns (5) + : : +- CometBroadcastExchange (14) + : : +- CometFilter (13) + : : +- CometScan parquet spark_catalog.default.warehouse (12) + : +- CometBroadcastExchange (20) + : +- CometProject (19) + : +- CometFilter (18) + : +- CometScan parquet spark_catalog.default.item (17) + +- CometBroadcastExchange (25) + +- CometFilter (24) + +- CometScan parquet spark_catalog.default.date_dim (23) (1) Scan parquet spark_catalog.default.catalog_sales @@ -46,173 +45,168 @@ ReadSchema: struct -(7) CometFilter +(6) CometFilter Input [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] Condition : (isnotnull(cr_order_number#8) AND isnotnull(cr_item_sk#7)) -(8) CometProject +(7) CometProject Input [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] Arguments: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9], [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -(9) ColumnarToRow [codegen id : 3] +(8) CometColumnarExchange Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] +Arguments: hashpartitioning(cr_order_number#8, cr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(10) Exchange +(9) CometSort Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: hashpartitioning(cr_order_number#8, cr_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9], [cr_order_number#8 ASC NULLS FIRST, cr_item_sk#7 ASC NULLS FIRST] -(11) Sort [codegen id : 4] -Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cr_order_number#8 ASC NULLS FIRST, cr_item_sk#7 ASC NULLS FIRST], false, 0 +(10) CometSortMergeJoin +Left output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] +Right output [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] +Arguments: [cs_order_number#3, cs_item_sk#2], [cr_order_number#8, cr_item_sk#7], LeftOuter -(12) SortMergeJoin [codegen id : 8] -Left keys [2]: [cs_order_number#3, cs_item_sk#2] -Right keys [2]: [cr_order_number#8, cr_item_sk#7] -Join type: LeftOuter -Join condition: None - -(13) Project [codegen id : 8] -Output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9] +(11) CometProject Input [8]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5, cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] +Arguments: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9], [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9] -(14) Scan parquet spark_catalog.default.warehouse +(12) Scan parquet spark_catalog.default.warehouse Output [2]: [w_warehouse_sk#11, w_state#12] Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] PushedFilters: [IsNotNull(w_warehouse_sk)] ReadSchema: struct -(15) CometFilter +(13) CometFilter Input [2]: [w_warehouse_sk#11, w_state#12] Condition : isnotnull(w_warehouse_sk#11) -(16) ColumnarToRow [codegen id : 5] -Input [2]: [w_warehouse_sk#11, w_state#12] - -(17) BroadcastExchange +(14) CometBroadcastExchange Input [2]: [w_warehouse_sk#11, w_state#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +Arguments: [w_warehouse_sk#11, w_state#12] -(18) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_warehouse_sk#1] -Right keys [1]: [w_warehouse_sk#11] -Join type: Inner -Join condition: None +(15) CometBroadcastHashJoin +Left output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9] +Right output [2]: [w_warehouse_sk#11, w_state#12] +Arguments: [cs_warehouse_sk#1], [w_warehouse_sk#11], Inner, BuildRight -(19) Project [codegen id : 8] -Output [5]: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12] +(16) CometProject Input [7]: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_warehouse_sk#11, w_state#12] +Arguments: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12], [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12] -(20) Scan parquet spark_catalog.default.item +(17) Scan parquet spark_catalog.default.item Output [3]: [i_item_sk#13, i_item_id#14, i_current_price#15] 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 -(21) CometFilter +(18) CometFilter Input [3]: [i_item_sk#13, i_item_id#14, i_current_price#15] Condition : (((isnotnull(i_current_price#15) AND (i_current_price#15 >= 0.99)) AND (i_current_price#15 <= 1.49)) AND isnotnull(i_item_sk#13)) -(22) CometProject +(19) CometProject Input [3]: [i_item_sk#13, i_item_id#14, i_current_price#15] Arguments: [i_item_sk#13, i_item_id#14], [i_item_sk#13, i_item_id#14] -(23) ColumnarToRow [codegen id : 6] +(20) CometBroadcastExchange Input [2]: [i_item_sk#13, i_item_id#14] +Arguments: [i_item_sk#13, i_item_id#14] -(24) BroadcastExchange -Input [2]: [i_item_sk#13, i_item_id#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(21) CometBroadcastHashJoin +Left output [5]: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12] +Right output [2]: [i_item_sk#13, i_item_id#14] +Arguments: [cs_item_sk#2], [i_item_sk#13], Inner, BuildRight -(25) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_item_sk#2] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None - -(26) Project [codegen id : 8] -Output [5]: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12, i_item_id#14] +(22) CometProject Input [7]: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12, i_item_sk#13, i_item_id#14] +Arguments: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12, i_item_id#14], [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12, i_item_id#14] -(27) ReusedExchange [Reuses operator id: 37] +(23) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#16, d_date#17] +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 + +(24) CometFilter +Input [2]: [d_date_sk#16, d_date#17] +Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 2000-02-10)) AND (d_date#17 <= 2000-04-10)) AND isnotnull(d_date_sk#16)) -(28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#16] -Join type: Inner -Join condition: None +(25) CometBroadcastExchange +Input [2]: [d_date_sk#16, d_date#17] +Arguments: [d_date_sk#16, d_date#17] -(29) Project [codegen id : 8] -Output [5]: [cs_sales_price#4, cr_refunded_cash#9, w_state#12, i_item_id#14, d_date#17] +(26) CometBroadcastHashJoin +Left output [5]: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12, i_item_id#14] +Right output [2]: [d_date_sk#16, d_date#17] +Arguments: [cs_sold_date_sk#5], [d_date_sk#16], Inner, BuildRight + +(27) CometProject Input [7]: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12, i_item_id#14, d_date_sk#16, d_date#17] +Arguments: [cs_sales_price#4, cr_refunded_cash#9, w_state#12, i_item_id#14, d_date#17], [cs_sales_price#4, cr_refunded_cash#9, w_state#12, i_item_id#14, d_date#17] -(30) HashAggregate [codegen id : 8] +(28) CometHashAggregate Input [5]: [cs_sales_price#4, cr_refunded_cash#9, w_state#12, i_item_id#14, d_date#17] Keys [2]: [w_state#12, i_item_id#14] Functions [2]: [partial_sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END), partial_sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)] -Aggregate Attributes [4]: [sum#18, isEmpty#19, sum#20, isEmpty#21] -Results [6]: [w_state#12, i_item_id#14, sum#22, isEmpty#23, sum#24, isEmpty#25] -(31) Exchange -Input [6]: [w_state#12, i_item_id#14, sum#22, isEmpty#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(w_state#12, i_item_id#14, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(29) CometColumnarExchange +Input [6]: [w_state#12, i_item_id#14, sum#18, isEmpty#19, sum#20, isEmpty#21] +Arguments: hashpartitioning(w_state#12, i_item_id#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(32) HashAggregate [codegen id : 9] -Input [6]: [w_state#12, i_item_id#14, sum#22, isEmpty#23, sum#24, isEmpty#25] +(30) CometHashAggregate +Input [6]: [w_state#12, i_item_id#14, sum#18, isEmpty#19, sum#20, isEmpty#21] Keys [2]: [w_state#12, i_item_id#14] Functions [2]: [sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END), sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)] -Aggregate Attributes [2]: [sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#26, sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#27] -Results [4]: [w_state#12, i_item_id#14, sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#26 AS sales_before#28, sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#27 AS sales_after#29] -(33) TakeOrderedAndProject -Input [4]: [w_state#12, i_item_id#14, sales_before#28, sales_after#29] -Arguments: 100, [w_state#12 ASC NULLS FIRST, i_item_id#14 ASC NULLS FIRST], [w_state#12, i_item_id#14, sales_before#28, sales_after#29] +(31) CometTakeOrderedAndProject +Input [4]: [w_state#12, i_item_id#14, sales_before#22, sales_after#23] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_state#12 ASC NULLS FIRST,i_item_id#14 ASC NULLS FIRST], output=[w_state#12,i_item_id#14,sales_before#22,sales_after#23]), [w_state#12, i_item_id#14, sales_before#22, sales_after#23], 100, [w_state#12 ASC NULLS FIRST, i_item_id#14 ASC NULLS FIRST], [w_state#12, i_item_id#14, sales_before#22, sales_after#23] + +(32) ColumnarToRow [codegen id : 1] +Input [4]: [w_state#12, i_item_id#14, sales_before#22, sales_after#23] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (37) -+- * ColumnarToRow (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.date_dim (34) +BroadcastExchange (36) ++- * ColumnarToRow (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) -(34) Scan parquet spark_catalog.default.date_dim +(33) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#16, d_date#17] 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 -(35) CometFilter +(34) CometFilter Input [2]: [d_date_sk#16, d_date#17] Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 2000-02-10)) AND (d_date#17 <= 2000-04-10)) AND isnotnull(d_date_sk#16)) -(36) ColumnarToRow [codegen id : 1] +(35) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#16, d_date#17] -(37) BroadcastExchange +(36) BroadcastExchange Input [2]: [d_date_sk#16, d_date#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt index 11c0201a8..6e47fa59f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt @@ -1,60 +1,41 @@ -TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] - WholeStageCodegen (9) - HashAggregate [w_state,i_item_id,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_date < 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END),sales_before,sales_after,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [w_state,i_item_id] #1 - WholeStageCodegen (8) - HashAggregate [w_state,i_item_id,d_date,cs_sales_price,cr_refunded_cash] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Project [cs_sales_price,cr_refunded_cash,w_state,i_item_id,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state] - BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - Project [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash] - SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - InputAdapter - WholeStageCodegen (2) - Sort [cs_order_number,cs_item_sk] - InputAdapter - Exchange [cs_order_number,cs_item_sk] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - WholeStageCodegen (4) - Sort [cr_order_number,cr_item_sk] - InputAdapter - Exchange [cr_order_number,cr_item_sk] #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk,w_state] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id,i_current_price] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] + CometHashAggregate [w_state,i_item_id,sales_before,sales_after,sum,isEmpty,sum,isEmpty,sum(CASE WHEN (d_date < 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END)] + CometColumnarExchange [w_state,i_item_id] #1 + CometHashAggregate [w_state,i_item_id,sum,isEmpty,sum,isEmpty,d_date,cs_sales_price,cr_refunded_cash] + CometProject [cs_sales_price,cr_refunded_cash,w_state,i_item_id,d_date] + CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_id,d_date_sk,d_date] + CometProject [cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_id] + CometBroadcastHashJoin [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_sk,i_item_id] + CometProject [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state] + CometBroadcastHashJoin [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_warehouse_sk,w_state] + CometProject [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash] + CometSortMergeJoin [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_refunded_cash] + CometSort [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] + CometColumnarExchange [cs_order_number,cs_item_sk] #2 + CometFilter [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [cr_item_sk,cr_order_number,cr_refunded_cash] + CometColumnarExchange [cr_order_number,cr_item_sk] #4 + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] + CometBroadcastExchange [w_warehouse_sk,w_state] #5 + CometFilter [w_warehouse_sk,w_state] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometProject [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id,i_current_price] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] + CometBroadcastExchange [d_date_sk,d_date] #7 + CometFilter [d_date_sk,d_date] + 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/q41/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt index 07196ba8c..5c2b28b9d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt @@ -1,24 +1,23 @@ == Physical Plan == -TakeOrderedAndProject (20) -+- * HashAggregate (19) - +- Exchange (18) - +- * HashAggregate (17) - +- * Project (16) - +- * BroadcastHashJoin Inner BuildRight (15) - :- * ColumnarToRow (4) - : +- CometProject (3) - : +- CometFilter (2) - : +- CometScan parquet spark_catalog.default.item (1) - +- BroadcastExchange (14) - +- * Project (13) - +- * Filter (12) - +- * HashAggregate (11) - +- Exchange (10) - +- * ColumnarToRow (9) - +- CometHashAggregate (8) - +- CometProject (7) - +- CometFilter (6) - +- CometScan parquet spark_catalog.default.item (5) +* ColumnarToRow (19) ++- CometTakeOrderedAndProject (18) + +- CometHashAggregate (17) + +- CometColumnarExchange (16) + +- CometHashAggregate (15) + +- CometProject (14) + +- CometBroadcastHashJoin (13) + :- CometProject (3) + : +- CometFilter (2) + : +- CometScan parquet spark_catalog.default.item (1) + +- CometBroadcastExchange (12) + +- CometProject (11) + +- CometFilter (10) + +- CometHashAggregate (9) + +- CometColumnarExchange (8) + +- CometHashAggregate (7) + +- CometProject (6) + +- CometFilter (5) + +- CometScan parquet spark_catalog.default.item (4) (1) Scan parquet spark_catalog.default.item @@ -36,84 +35,74 @@ Condition : (((isnotnull(i_manufact_id#1) AND (i_manufact_id#1 >= 738)) AND (i_m Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] Arguments: [i_manufact#2, i_product_name#3], [i_manufact#2, i_product_name#3] -(4) ColumnarToRow [codegen id : 3] -Input [2]: [i_manufact#2, i_product_name#3] - -(5) Scan parquet spark_catalog.default.item +(4) Scan parquet spark_catalog.default.item Output [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [Or(Or(And(EqualTo(i_category,Women ),Or(And(And(Or(EqualTo(i_color,powder ),EqualTo(i_color,khaki )),Or(EqualTo(i_units,Ounce ),EqualTo(i_units,Oz ))),Or(EqualTo(i_size,medium ),EqualTo(i_size,extra large ))),And(And(Or(EqualTo(i_color,brown ),EqualTo(i_color,honeydew )),Or(EqualTo(i_units,Bunch ),EqualTo(i_units,Ton ))),Or(EqualTo(i_size,N/A ),EqualTo(i_size,small ))))),And(EqualTo(i_category,Men ),Or(And(And(Or(EqualTo(i_color,floral ),EqualTo(i_color,deep )),Or(EqualTo(i_units,N/A ),EqualTo(i_units,Dozen ))),Or(EqualTo(i_size,petite ),EqualTo(i_size,large ))),And(And(Or(EqualTo(i_color,light ),EqualTo(i_color,cornflower )),Or(EqualTo(i_units,Box ),EqualTo(i_units,Pound ))),Or(EqualTo(i_size,medium ),EqualTo(i_size,extra large )))))),Or(And(EqualTo(i_category,Women ),Or(And(And(Or(EqualTo(i_color,midnight ),EqualTo(i_color,snow )),Or(EqualTo(i_units,Pallet ),EqualTo(i_units,Gross ))),Or(EqualTo(i_size,medium ),EqualTo(i_size,extra large ))),And(And(Or(EqualTo(i_color,cyan ),EqualTo(i_color,papaya )),Or(EqualTo(i_units,Cup ),EqualTo(i_units,Dram ))),Or(EqualTo(i_size,N/A ),EqualTo(i_size,small ))))),And(EqualTo(i_category,Men ),Or(And(And(Or(EqualTo(i_color,orange ),EqualTo(i_color,frosted )),Or(EqualTo(i_units,Each ),EqualTo(i_units,Tbl ))),Or(EqualTo(i_size,petite ),EqualTo(i_size,large ))),And(And(Or(EqualTo(i_color,forest ),EqualTo(i_color,ghost )),Or(EqualTo(i_units,Lb ),EqualTo(i_units,Bundle ))),Or(EqualTo(i_size,medium ),EqualTo(i_size,extra large ))))))), IsNotNull(i_manufact)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] Condition : (((((i_category#4 = Women ) AND (((((i_color#7 = powder ) OR (i_color#7 = khaki )) AND ((i_units#8 = Ounce ) OR (i_units#8 = Oz ))) AND ((i_size#6 = medium ) OR (i_size#6 = extra large ))) OR ((((i_color#7 = brown ) OR (i_color#7 = honeydew )) AND ((i_units#8 = Bunch ) OR (i_units#8 = Ton ))) AND ((i_size#6 = N/A ) OR (i_size#6 = small ))))) OR ((i_category#4 = Men ) AND (((((i_color#7 = floral ) OR (i_color#7 = deep )) AND ((i_units#8 = N/A ) OR (i_units#8 = Dozen ))) AND ((i_size#6 = petite ) OR (i_size#6 = large ))) OR ((((i_color#7 = light ) OR (i_color#7 = cornflower )) AND ((i_units#8 = Box ) OR (i_units#8 = Pound ))) AND ((i_size#6 = medium ) OR (i_size#6 = extra large )))))) OR (((i_category#4 = Women ) AND (((((i_color#7 = midnight ) OR (i_color#7 = snow )) AND ((i_units#8 = Pallet ) OR (i_units#8 = Gross ))) AND ((i_size#6 = medium ) OR (i_size#6 = extra large ))) OR ((((i_color#7 = cyan ) OR (i_color#7 = papaya )) AND ((i_units#8 = Cup ) OR (i_units#8 = Dram ))) AND ((i_size#6 = N/A ) OR (i_size#6 = small ))))) OR ((i_category#4 = Men ) AND (((((i_color#7 = orange ) OR (i_color#7 = frosted )) AND ((i_units#8 = Each ) OR (i_units#8 = Tbl ))) AND ((i_size#6 = petite ) OR (i_size#6 = large ))) OR ((((i_color#7 = forest ) OR (i_color#7 = ghost )) AND ((i_units#8 = Lb ) OR (i_units#8 = Bundle ))) AND ((i_size#6 = medium ) OR (i_size#6 = extra large ))))))) AND isnotnull(i_manufact#5)) -(7) CometProject +(6) CometProject Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] Arguments: [i_manufact#5], [i_manufact#5] -(8) CometHashAggregate +(7) CometHashAggregate Input [1]: [i_manufact#5] Keys [1]: [i_manufact#5] Functions [1]: [partial_count(1)] -(9) ColumnarToRow [codegen id : 1] -Input [2]: [i_manufact#5, count#9] - -(10) Exchange +(8) CometColumnarExchange Input [2]: [i_manufact#5, count#9] -Arguments: hashpartitioning(i_manufact#5, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: hashpartitioning(i_manufact#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(11) HashAggregate [codegen id : 2] +(9) CometHashAggregate Input [2]: [i_manufact#5, count#9] Keys [1]: [i_manufact#5] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#10] -Results [2]: [count(1)#10 AS item_cnt#11, i_manufact#5] -(12) Filter [codegen id : 2] -Input [2]: [item_cnt#11, i_manufact#5] -Condition : (item_cnt#11 > 0) +(10) CometFilter +Input [2]: [item_cnt#10, i_manufact#5] +Condition : (item_cnt#10 > 0) -(13) Project [codegen id : 2] -Output [1]: [i_manufact#5] -Input [2]: [item_cnt#11, i_manufact#5] +(11) CometProject +Input [2]: [item_cnt#10, i_manufact#5] +Arguments: [i_manufact#5], [i_manufact#5] -(14) BroadcastExchange +(12) CometBroadcastExchange Input [1]: [i_manufact#5] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] +Arguments: [i_manufact#5] -(15) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [i_manufact#2] -Right keys [1]: [i_manufact#5] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [2]: [i_manufact#2, i_product_name#3] +Right output [1]: [i_manufact#5] +Arguments: [i_manufact#2], [i_manufact#5], Inner, BuildRight -(16) Project [codegen id : 3] -Output [1]: [i_product_name#3] +(14) CometProject Input [3]: [i_manufact#2, i_product_name#3, i_manufact#5] +Arguments: [i_product_name#3], [i_product_name#3] -(17) HashAggregate [codegen id : 3] +(15) CometHashAggregate Input [1]: [i_product_name#3] Keys [1]: [i_product_name#3] Functions: [] -Aggregate Attributes: [] -Results [1]: [i_product_name#3] -(18) Exchange +(16) CometColumnarExchange Input [1]: [i_product_name#3] -Arguments: hashpartitioning(i_product_name#3, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(i_product_name#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(19) HashAggregate [codegen id : 4] +(17) CometHashAggregate Input [1]: [i_product_name#3] Keys [1]: [i_product_name#3] Functions: [] -Aggregate Attributes: [] -Results [1]: [i_product_name#3] -(20) TakeOrderedAndProject +(18) CometTakeOrderedAndProject +Input [1]: [i_product_name#3] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_product_name#3 ASC NULLS FIRST], output=[i_product_name#3]), [i_product_name#3], 100, [i_product_name#3 ASC NULLS FIRST], [i_product_name#3] + +(19) ColumnarToRow [codegen id : 1] Input [1]: [i_product_name#3] -Arguments: 100, [i_product_name#3 ASC NULLS FIRST], [i_product_name#3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt index 457d441ce..732e8679b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt @@ -1,29 +1,21 @@ -TakeOrderedAndProject [i_product_name] - WholeStageCodegen (4) - HashAggregate [i_product_name] - InputAdapter - Exchange [i_product_name] #1 - WholeStageCodegen (3) - HashAggregate [i_product_name] - Project [i_product_name] - BroadcastHashJoin [i_manufact,i_manufact] - ColumnarToRow - InputAdapter - CometProject [i_manufact,i_product_name] - CometFilter [i_manufact_id,i_manufact,i_product_name] - CometScan parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [i_manufact] - Filter [item_cnt] - HashAggregate [i_manufact,count] [count(1),item_cnt,count] - InputAdapter - Exchange [i_manufact] #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_manufact,count] - CometProject [i_manufact] - CometFilter [i_category,i_manufact,i_size,i_color,i_units] - CometScan parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_product_name] + CometHashAggregate [i_product_name] + CometColumnarExchange [i_product_name] #1 + CometHashAggregate [i_product_name] + CometProject [i_product_name] + CometBroadcastHashJoin [i_manufact,i_product_name,i_manufact] + CometProject [i_manufact,i_product_name] + CometFilter [i_manufact_id,i_manufact,i_product_name] + CometScan parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] + CometBroadcastExchange [i_manufact] #2 + CometProject [i_manufact] + CometFilter [item_cnt,i_manufact] + CometHashAggregate [item_cnt,i_manufact,count,count(1)] + CometColumnarExchange [i_manufact] #3 + CometHashAggregate [i_manufact,count] + CometProject [i_manufact] + CometFilter [i_category,i_manufact,i_size,i_color,i_units] + CometScan parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] 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 565b30e9a..ae0d6f333 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,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (19) -+- * HashAggregate (18) - +- Exchange (17) - +- * ColumnarToRow (16) +* ColumnarToRow (19) ++- CometTakeOrderedAndProject (18) + +- CometHashAggregate (17) + +- CometColumnarExchange (16) +- CometHashAggregate (15) +- CometProject (14) +- CometBroadcastHashJoin (13) @@ -93,21 +93,19 @@ 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))] -(16) ColumnarToRow [codegen id : 1] +(16) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(18) HashAggregate [codegen id : 2] +(17) CometHashAggregate 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))#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] -(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] +(18) CometTakeOrderedAndProject +Input [4]: [d_year#2, i_category_id#8, i_category#9, sum(ss_ext_sales_price)#12] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[sum(ss_ext_sales_price)#12 DESC NULLS LAST,d_year#2 ASC NULLS FIRST,i_category_id#8 ASC NULLS FIRST,i_category#9 ASC NULLS FIRST], output=[d_year#2,i_category_id#8,i_category#9,sum(ss_ext_sales_price)#12]), [d_year#2, i_category_id#8, i_category#9, sum(ss_ext_sales_price)#12], 100, [sum(ss_ext_sales_price)#12 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)#12] + +(19) ColumnarToRow [codegen id : 1] +Input [4]: [d_year#2, i_category_id#8, i_category#9, sum(ss_ext_sales_price)#12] 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 e3b850ef5..bdc71f6da 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,23 +1,21 @@ -TakeOrderedAndProject [sum(ss_ext_sales_price),d_year,i_category_id,i_category] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [d_year,i_category_id,i_category,sum,ss_ext_sales_price] - CometProject [d_year,ss_ext_sales_price,i_category_id,i_category] - CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_category_id,i_category] - CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_category_id,i_category] #3 - CometProject [i_item_sk,i_category_id,i_category] - CometFilter [i_item_sk,i_category_id,i_category,i_manager_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_category_id,i_category,i_manager_id] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [d_year,i_category_id,i_category,sum(ss_ext_sales_price)] + CometHashAggregate [d_year,i_category_id,i_category,sum(ss_ext_sales_price),sum,sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange [d_year,i_category_id,i_category] #1 + CometHashAggregate [d_year,i_category_id,i_category,sum,ss_ext_sales_price] + CometProject [d_year,ss_ext_sales_price,i_category_id,i_category] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_category_id,i_category] + CometProject [d_year,ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometProject [d_date_sk,d_year] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange [i_item_sk,i_category_id,i_category] #3 + CometProject [i_item_sk,i_category_id,i_category] + CometFilter [i_item_sk,i_category_id,i_category,i_manager_id] + 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 a2cfe7898..41a564299 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,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (19) -+- * HashAggregate (18) - +- Exchange (17) - +- * ColumnarToRow (16) +* ColumnarToRow (19) ++- CometTakeOrderedAndProject (18) + +- CometHashAggregate (17) + +- CometColumnarExchange (16) +- CometHashAggregate (15) +- CometProject (14) +- CometBroadcastHashJoin (13) @@ -93,21 +93,19 @@ 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))] -(16) ColumnarToRow [codegen id : 1] +(16) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(18) HashAggregate [codegen id : 2] +(17) CometHashAggregate 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))#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] -(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] +(18) CometTakeOrderedAndProject +Input [9]: [s_store_name#9, s_store_id#8, sun_sales#18, mon_sales#19, tue_sales#20, wed_sales#21, thu_sales#22, fri_sales#23, sat_sales#24] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#9 ASC NULLS FIRST,s_store_id#8 ASC NULLS FIRST,sun_sales#18 ASC NULLS FIRST,mon_sales#19 ASC NULLS FIRST,tue_sales#20 ASC NULLS FIRST,wed_sales#21 ASC NULLS FIRST,thu_sales#22 ASC NULLS FIRST,fri_sales#23 ASC NULLS FIRST,sat_sales#24 ASC NULLS FIRST], output=[s_store_name#9,s_store_id#8,sun_sales#18,mon_sales#19,tue_sales#20,wed_sales#21,thu_sales#22,fri_sales#23,sat_sales#24]), [s_store_name#9, s_store_id#8, sun_sales#18, mon_sales#19, tue_sales#20, wed_sales#21, thu_sales#22, fri_sales#23, sat_sales#24], 100, [s_store_name#9 ASC NULLS FIRST, s_store_id#8 ASC NULLS FIRST, sun_sales#18 ASC NULLS FIRST, mon_sales#19 ASC NULLS FIRST, tue_sales#20 ASC NULLS FIRST, wed_sales#21 ASC NULLS FIRST, thu_sales#22 ASC NULLS FIRST, fri_sales#23 ASC NULLS FIRST, sat_sales#24 ASC NULLS FIRST], [s_store_name#9, s_store_id#8, sun_sales#18, mon_sales#19, tue_sales#20, wed_sales#21, thu_sales#22, fri_sales#23, sat_sales#24] + +(19) ColumnarToRow [codegen id : 1] +Input [9]: [s_store_name#9, s_store_id#8, sun_sales#18, mon_sales#19, tue_sales#20, wed_sales#21, thu_sales#22, fri_sales#23, sat_sales#24] 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 5a060b85c..6e1b8d6bf 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,23 +1,21 @@ -TakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum,d_day_name,ss_sales_price] - CometProject [d_day_name,ss_sales_price,s_store_id,s_store_name] - CometBroadcastHashJoin [d_day_name,ss_store_sk,ss_sales_price,s_store_sk,s_store_id,s_store_name] - CometProject [d_day_name,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [d_date_sk,d_day_name,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_day_name] - CometFilter [d_date_sk,d_year,d_day_name] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] - CometBroadcastExchange [ss_store_sk,ss_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 - CometProject [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name,s_gmt_offset] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] + CometHashAggregate [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,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))] + CometColumnarExchange [s_store_name,s_store_id] #1 + CometHashAggregate [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum,d_day_name,ss_sales_price] + CometProject [d_day_name,ss_sales_price,s_store_id,s_store_name] + CometBroadcastHashJoin [d_day_name,ss_store_sk,ss_sales_price,s_store_sk,s_store_id,s_store_name] + CometProject [d_day_name,ss_store_sk,ss_sales_price] + CometBroadcastHashJoin [d_date_sk,d_day_name,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [d_date_sk,d_day_name] + CometFilter [d_date_sk,d_year,d_day_name] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] + CometBroadcastExchange [ss_store_sk,ss_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 + CometProject [s_store_sk,s_store_id,s_store_name] + CometFilter [s_store_sk,s_store_id,s_store_name,s_gmt_offset] + 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/q44/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt index 8408eab0e..af6264478 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt @@ -1,36 +1,37 @@ == Physical Plan == -TakeOrderedAndProject (32) -+- * Project (31) - +- * BroadcastHashJoin Inner BuildRight (30) - :- * Project (28) - : +- * BroadcastHashJoin Inner BuildRight (27) - : :- * Project (22) - : : +- * SortMergeJoin Inner (21) +TakeOrderedAndProject (33) ++- * Project (32) + +- * BroadcastHashJoin Inner BuildRight (31) + :- * Project (29) + : +- * BroadcastHashJoin Inner BuildRight (28) + : :- * Project (23) + : : +- * SortMergeJoin Inner (22) : : :- * Sort (14) : : : +- * Project (13) : : : +- * Filter (12) : : : +- Window (11) - : : : +- * Sort (10) - : : : +- Exchange (9) - : : : +- * Filter (8) - : : : +- * HashAggregate (7) - : : : +- Exchange (6) - : : : +- * ColumnarToRow (5) + : : : +- * ColumnarToRow (10) + : : : +- CometSort (9) + : : : +- CometColumnarExchange (8) + : : : +- CometFilter (7) + : : : +- CometHashAggregate (6) + : : : +- CometColumnarExchange (5) : : : +- CometHashAggregate (4) : : : +- CometProject (3) : : : +- CometFilter (2) : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : +- * Sort (20) - : : +- * Project (19) - : : +- * Filter (18) - : : +- Window (17) - : : +- * Sort (16) - : : +- ReusedExchange (15) - : +- BroadcastExchange (26) - : +- * ColumnarToRow (25) - : +- CometFilter (24) - : +- CometScan parquet spark_catalog.default.item (23) - +- ReusedExchange (29) + : : +- * Sort (21) + : : +- * Project (20) + : : +- * Filter (19) + : : +- Window (18) + : : +- * ColumnarToRow (17) + : : +- CometSort (16) + : : +- ReusedExchange (15) + : +- BroadcastExchange (27) + : +- * ColumnarToRow (26) + : +- CometFilter (25) + : +- CometScan parquet spark_catalog.default.item (24) + +- ReusedExchange (30) (1) Scan parquet spark_catalog.default.store_sales @@ -53,170 +54,169 @@ Input [2]: [ss_item_sk#1, ss_net_profit#3] Keys [1]: [ss_item_sk#1] Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#3))] -(5) ColumnarToRow [codegen id : 1] +(5) CometColumnarExchange Input [3]: [ss_item_sk#1, sum#5, count#6] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(6) Exchange -Input [3]: [ss_item_sk#1, sum#5, count#6] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] - -(7) HashAggregate [codegen id : 2] +(6) CometHashAggregate Input [3]: [ss_item_sk#1, sum#5, count#6] Keys [1]: [ss_item_sk#1] Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#3))#7] -Results [2]: [ss_item_sk#1 AS item_sk#8, cast((avg(UnscaledValue(ss_net_profit#3))#7 / 100.0) as decimal(11,6)) AS rank_col#9] -(8) Filter [codegen id : 2] -Input [2]: [item_sk#8, rank_col#9] -Condition : (isnotnull(rank_col#9) AND (cast(rank_col#9 as decimal(13,7)) > (0.9 * Subquery scalar-subquery#10, [id=#11]))) +(7) CometFilter +Input [2]: [item_sk#7, rank_col#8] +Condition : (isnotnull(rank_col#8) AND (cast(rank_col#8 as decimal(13,7)) > (0.9 * Subquery scalar-subquery#9, [id=#10]))) -(9) Exchange -Input [2]: [item_sk#8, rank_col#9] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] +(8) CometColumnarExchange +Input [2]: [item_sk#7, rank_col#8] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(10) Sort [codegen id : 3] -Input [2]: [item_sk#8, rank_col#9] -Arguments: [rank_col#9 ASC NULLS FIRST], false, 0 +(9) CometSort +Input [2]: [item_sk#7, rank_col#8] +Arguments: [item_sk#7, rank_col#8], [rank_col#8 ASC NULLS FIRST] + +(10) ColumnarToRow [codegen id : 1] +Input [2]: [item_sk#7, rank_col#8] (11) Window -Input [2]: [item_sk#8, rank_col#9] -Arguments: [rank(rank_col#9) windowspecdefinition(rank_col#9 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#12], [rank_col#9 ASC NULLS FIRST] +Input [2]: [item_sk#7, rank_col#8] +Arguments: [rank(rank_col#8) windowspecdefinition(rank_col#8 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#11], [rank_col#8 ASC NULLS FIRST] + +(12) Filter [codegen id : 2] +Input [3]: [item_sk#7, rank_col#8, rnk#11] +Condition : ((rnk#11 < 11) AND isnotnull(item_sk#7)) -(12) Filter [codegen id : 4] -Input [3]: [item_sk#8, rank_col#9, rnk#12] -Condition : ((rnk#12 < 11) AND isnotnull(item_sk#8)) +(13) Project [codegen id : 2] +Output [2]: [item_sk#7, rnk#11] +Input [3]: [item_sk#7, rank_col#8, rnk#11] -(13) Project [codegen id : 4] -Output [2]: [item_sk#8, rnk#12] -Input [3]: [item_sk#8, rank_col#9, rnk#12] +(14) Sort [codegen id : 2] +Input [2]: [item_sk#7, rnk#11] +Arguments: [rnk#11 ASC NULLS FIRST], false, 0 -(14) Sort [codegen id : 4] -Input [2]: [item_sk#8, rnk#12] -Arguments: [rnk#12 ASC NULLS FIRST], false, 0 +(15) ReusedExchange [Reuses operator id: 8] +Output [2]: [item_sk#12, rank_col#13] -(15) ReusedExchange [Reuses operator id: 9] -Output [2]: [item_sk#13, rank_col#14] +(16) CometSort +Input [2]: [item_sk#12, rank_col#13] +Arguments: [item_sk#12, rank_col#13], [rank_col#13 DESC NULLS LAST] -(16) Sort [codegen id : 7] -Input [2]: [item_sk#13, rank_col#14] -Arguments: [rank_col#14 DESC NULLS LAST], false, 0 +(17) ColumnarToRow [codegen id : 3] +Input [2]: [item_sk#12, rank_col#13] -(17) Window -Input [2]: [item_sk#13, rank_col#14] -Arguments: [rank(rank_col#14) windowspecdefinition(rank_col#14 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#15], [rank_col#14 DESC NULLS LAST] +(18) Window +Input [2]: [item_sk#12, rank_col#13] +Arguments: [rank(rank_col#13) windowspecdefinition(rank_col#13 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#14], [rank_col#13 DESC NULLS LAST] -(18) Filter [codegen id : 8] -Input [3]: [item_sk#13, rank_col#14, rnk#15] -Condition : ((rnk#15 < 11) AND isnotnull(item_sk#13)) +(19) Filter [codegen id : 4] +Input [3]: [item_sk#12, rank_col#13, rnk#14] +Condition : ((rnk#14 < 11) AND isnotnull(item_sk#12)) -(19) Project [codegen id : 8] -Output [2]: [item_sk#13, rnk#15] -Input [3]: [item_sk#13, rank_col#14, rnk#15] +(20) Project [codegen id : 4] +Output [2]: [item_sk#12, rnk#14] +Input [3]: [item_sk#12, rank_col#13, rnk#14] -(20) Sort [codegen id : 8] -Input [2]: [item_sk#13, rnk#15] -Arguments: [rnk#15 ASC NULLS FIRST], false, 0 +(21) Sort [codegen id : 4] +Input [2]: [item_sk#12, rnk#14] +Arguments: [rnk#14 ASC NULLS FIRST], false, 0 -(21) SortMergeJoin [codegen id : 11] -Left keys [1]: [rnk#12] -Right keys [1]: [rnk#15] +(22) SortMergeJoin [codegen id : 7] +Left keys [1]: [rnk#11] +Right keys [1]: [rnk#14] Join type: Inner Join condition: None -(22) Project [codegen id : 11] -Output [3]: [item_sk#8, rnk#12, item_sk#13] -Input [4]: [item_sk#8, rnk#12, item_sk#13, rnk#15] +(23) Project [codegen id : 7] +Output [3]: [item_sk#7, rnk#11, item_sk#12] +Input [4]: [item_sk#7, rnk#11, item_sk#12, rnk#14] -(23) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_product_name#17] +(24) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#15, i_product_name#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(24) CometFilter -Input [2]: [i_item_sk#16, i_product_name#17] -Condition : isnotnull(i_item_sk#16) +(25) CometFilter +Input [2]: [i_item_sk#15, i_product_name#16] +Condition : isnotnull(i_item_sk#15) -(25) ColumnarToRow [codegen id : 9] -Input [2]: [i_item_sk#16, i_product_name#17] +(26) ColumnarToRow [codegen id : 5] +Input [2]: [i_item_sk#15, i_product_name#16] -(26) BroadcastExchange -Input [2]: [i_item_sk#16, i_product_name#17] +(27) BroadcastExchange +Input [2]: [i_item_sk#15, i_product_name#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(27) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [item_sk#8] -Right keys [1]: [i_item_sk#16] +(28) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [item_sk#7] +Right keys [1]: [i_item_sk#15] Join type: Inner Join condition: None -(28) Project [codegen id : 11] -Output [3]: [rnk#12, item_sk#13, i_product_name#17] -Input [5]: [item_sk#8, rnk#12, item_sk#13, i_item_sk#16, i_product_name#17] +(29) Project [codegen id : 7] +Output [3]: [rnk#11, item_sk#12, i_product_name#16] +Input [5]: [item_sk#7, rnk#11, item_sk#12, i_item_sk#15, i_product_name#16] -(29) ReusedExchange [Reuses operator id: 26] -Output [2]: [i_item_sk#18, i_product_name#19] +(30) ReusedExchange [Reuses operator id: 27] +Output [2]: [i_item_sk#17, i_product_name#18] -(30) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [item_sk#13] -Right keys [1]: [i_item_sk#18] +(31) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [item_sk#12] +Right keys [1]: [i_item_sk#17] Join type: Inner Join condition: None -(31) Project [codegen id : 11] -Output [3]: [rnk#12, i_product_name#17 AS best_performing#20, i_product_name#19 AS worst_performing#21] -Input [5]: [rnk#12, item_sk#13, i_product_name#17, i_item_sk#18, i_product_name#19] +(32) Project [codegen id : 7] +Output [3]: [rnk#11, i_product_name#16 AS best_performing#19, i_product_name#18 AS worst_performing#20] +Input [5]: [rnk#11, item_sk#12, i_product_name#16, i_item_sk#17, i_product_name#18] -(32) TakeOrderedAndProject -Input [3]: [rnk#12, best_performing#20, worst_performing#21] -Arguments: 100, [rnk#12 ASC NULLS FIRST], [rnk#12, best_performing#20, worst_performing#21] +(33) TakeOrderedAndProject +Input [3]: [rnk#11, best_performing#19, worst_performing#20] +Arguments: 100, [rnk#11 ASC NULLS FIRST], [rnk#11, best_performing#19, worst_performing#20] ===== Subqueries ===== -Subquery:1 Hosting operator id = 8 Hosting Expression = Subquery scalar-subquery#10, [id=#11] -* HashAggregate (39) -+- Exchange (38) - +- * ColumnarToRow (37) - +- CometHashAggregate (36) - +- CometProject (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.store_sales (33) +Subquery:1 Hosting operator id = 7 Hosting Expression = Subquery scalar-subquery#9, [id=#10] +* ColumnarToRow (40) ++- CometHashAggregate (39) + +- CometColumnarExchange (38) + +- CometHashAggregate (37) + +- CometProject (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.store_sales (34) -(33) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_addr_sk#22, ss_store_sk#23, ss_net_profit#24, ss_sold_date_sk#25] +(34) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_addr_sk#21, ss_store_sk#22, ss_net_profit#23, ss_sold_date_sk#24] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)] ReadSchema: struct -(34) CometFilter -Input [4]: [ss_addr_sk#22, ss_store_sk#23, ss_net_profit#24, ss_sold_date_sk#25] -Condition : ((isnotnull(ss_store_sk#23) AND (ss_store_sk#23 = 4)) AND isnull(ss_addr_sk#22)) +(35) CometFilter +Input [4]: [ss_addr_sk#21, ss_store_sk#22, ss_net_profit#23, ss_sold_date_sk#24] +Condition : ((isnotnull(ss_store_sk#22) AND (ss_store_sk#22 = 4)) AND isnull(ss_addr_sk#21)) -(35) CometProject -Input [4]: [ss_addr_sk#22, ss_store_sk#23, ss_net_profit#24, ss_sold_date_sk#25] -Arguments: [ss_store_sk#23, ss_net_profit#24], [ss_store_sk#23, ss_net_profit#24] +(36) CometProject +Input [4]: [ss_addr_sk#21, ss_store_sk#22, ss_net_profit#23, ss_sold_date_sk#24] +Arguments: [ss_store_sk#22, ss_net_profit#23], [ss_store_sk#22, ss_net_profit#23] -(36) CometHashAggregate -Input [2]: [ss_store_sk#23, ss_net_profit#24] -Keys [1]: [ss_store_sk#23] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#24))] +(37) CometHashAggregate +Input [2]: [ss_store_sk#22, ss_net_profit#23] +Keys [1]: [ss_store_sk#22] +Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#23))] -(37) ColumnarToRow [codegen id : 1] -Input [3]: [ss_store_sk#23, sum#26, count#27] +(38) CometColumnarExchange +Input [3]: [ss_store_sk#22, sum#25, count#26] +Arguments: hashpartitioning(ss_store_sk#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(38) Exchange -Input [3]: [ss_store_sk#23, sum#26, count#27] -Arguments: hashpartitioning(ss_store_sk#23, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(39) CometHashAggregate +Input [3]: [ss_store_sk#22, sum#25, count#26] +Keys [1]: [ss_store_sk#22] +Functions [1]: [avg(UnscaledValue(ss_net_profit#23))] -(39) HashAggregate [codegen id : 2] -Input [3]: [ss_store_sk#23, sum#26, count#27] -Keys [1]: [ss_store_sk#23] -Functions [1]: [avg(UnscaledValue(ss_net_profit#24))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#24))#28] -Results [1]: [cast((avg(UnscaledValue(ss_net_profit#24))#28 / 100.0) as decimal(11,6)) AS rank_col#29] +(40) ColumnarToRow [codegen id : 1] +Input [1]: [rank_col#27] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt index f713176f1..acb8001a1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] - WholeStageCodegen (11) + WholeStageCodegen (7) Project [rnk,i_product_name,i_product_name] BroadcastHashJoin [item_sk,i_item_sk] Project [rnk,item_sk,i_product_name] @@ -7,54 +7,49 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] Project [item_sk,rnk,item_sk] SortMergeJoin [rnk,rnk] InputAdapter - WholeStageCodegen (4) + WholeStageCodegen (2) Sort [rnk] Project [item_sk,rnk] Filter [rnk,item_sk] InputAdapter Window [rank_col] - WholeStageCodegen (3) - Sort [rank_col] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (2) - Filter [rank_col] + CometSort [item_sk,rank_col] + CometColumnarExchange #1 + CometFilter [item_sk,rank_col] Subquery #1 - WholeStageCodegen (2) - HashAggregate [ss_store_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),rank_col,sum,count] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [ss_store_sk] #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ss_store_sk,sum,count,ss_net_profit] - CometProject [ss_store_sk,ss_net_profit] - CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,sum,count] - InputAdapter - Exchange [ss_item_sk] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ss_item_sk,sum,count,ss_net_profit] - CometProject [ss_item_sk,ss_net_profit] - CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometHashAggregate [rank_col,ss_store_sk,sum,count,avg(UnscaledValue(ss_net_profit))] + CometColumnarExchange [ss_store_sk] #3 + CometHashAggregate [ss_store_sk,sum,count,ss_net_profit] + CometProject [ss_store_sk,ss_net_profit] + CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometHashAggregate [item_sk,rank_col,ss_item_sk,sum,count,avg(UnscaledValue(ss_net_profit))] + CometColumnarExchange [ss_item_sk] #2 + CometHashAggregate [ss_item_sk,sum,count,ss_net_profit] + CometProject [ss_item_sk,ss_net_profit] + CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] InputAdapter - WholeStageCodegen (8) + WholeStageCodegen (4) Sort [rnk] Project [item_sk,rnk] Filter [rnk,item_sk] InputAdapter Window [rank_col] - WholeStageCodegen (7) - Sort [rank_col] + WholeStageCodegen (3) + ColumnarToRow InputAdapter - ReusedExchange [item_sk,rank_col] #1 + CometSort [item_sk,rank_col] + ReusedExchange [item_sk,rank_col] #1 InputAdapter BroadcastExchange #4 - WholeStageCodegen (9) + WholeStageCodegen (5) ColumnarToRow InputAdapter CometFilter [i_item_sk,i_product_name] 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 f128499e3..044247540 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 @@ -1,40 +1,42 @@ == Physical Plan == -TakeOrderedAndProject (36) -+- * HashAggregate (35) - +- Exchange (34) - +- * HashAggregate (33) - +- * Project (32) - +- * Filter (31) - +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (30) - :- * 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) - +- * ColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan parquet spark_catalog.default.item (25) +TakeOrderedAndProject (38) ++- * HashAggregate (37) + +- * ColumnarToRow (36) + +- CometColumnarExchange (35) + +- RowToColumnar (34) + +- * HashAggregate (33) + +- * Project (32) + +- * Filter (31) + +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (30) + :- * 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) + +- * ColumnarToRow (28) + +- CometProject (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.item (25) (1) Scan parquet spark_catalog.default.web_sales @@ -195,50 +197,56 @@ Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] Aggregate Attributes [1]: [sum#19] Results [3]: [ca_zip#11, ca_city#10, sum#20] -(34) Exchange +(34) RowToColumnar 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 : 3] +(35) CometColumnarExchange +Input [3]: [ca_zip#11, ca_city#10, sum#20] +Arguments: hashpartitioning(ca_zip#11, ca_city#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(36) ColumnarToRow [codegen id : 3] +Input [3]: [ca_zip#11, ca_city#10, sum#20] + +(37) 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))#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 +(38) TakeOrderedAndProject 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 ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (41) -+- * ColumnarToRow (40) - +- CometProject (39) - +- CometFilter (38) - +- CometScan parquet spark_catalog.default.date_dim (37) +BroadcastExchange (43) ++- * ColumnarToRow (42) + +- CometProject (41) + +- CometFilter (40) + +- CometScan parquet spark_catalog.default.date_dim (39) -(37) Scan parquet spark_catalog.default.date_dim +(39) 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 -(38) CometFilter +(40) 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)) -(39) CometProject +(41) CometProject 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] +(42) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#12] -(41) BroadcastExchange +(43) BroadcastExchange Input [1]: [d_date_sk#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/q45/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt index f60fdb18a..8550b91c6 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,51 +1,53 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] 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 (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] - ColumnarToRow - InputAdapter - CometProject [ws_sales_price,ca_city,ca_zip,i_item_id] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ca_city,ca_zip,i_item_sk,i_item_id] - CometProject [ws_item_sk,ws_sales_price,ca_city,ca_zip] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip,d_date_sk] - CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_city,ca_zip] - CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk,c_customer_sk,c_current_addr_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_city,ca_zip] #4 - CometFilter [ca_address_sk,ca_city,ca_zip] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + ColumnarToRow + InputAdapter + CometColumnarExchange [ca_zip,ca_city] #1 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometProject [ws_sales_price,ca_city,ca_zip,i_item_id] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ca_city,ca_zip,i_item_sk,i_item_id] + CometProject [ws_item_sk,ws_sales_price,ca_city,ca_zip] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip,d_date_sk] + CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_city,ca_zip] + CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk,c_customer_sk,c_current_addr_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange [ca_address_sk,ca_city,ca_zip] #4 + CometFilter [ca_address_sk,ca_city,ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [i_item_id] + CometFilter [i_item_sk,i_item_id] + 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/q46/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt index fb99351a5..b81931528 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,46 +1,42 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * Project (41) - +- * BroadcastHashJoin Inner BuildRight (40) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * HashAggregate (29) - : : +- Exchange (28) - : : +- * 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) - +- BroadcastExchange (39) - +- * ColumnarToRow (38) - +- CometFilter (37) - +- CometScan parquet spark_catalog.default.customer_address (36) +* ColumnarToRow (38) ++- CometTakeOrderedAndProject (37) + +- CometProject (36) + +- CometBroadcastHashJoin (35) + :- CometProject (33) + : +- CometBroadcastHashJoin (32) + : :- CometHashAggregate (28) + : : +- CometColumnarExchange (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) + : +- CometBroadcastExchange (31) + : +- CometFilter (30) + : +- CometScan parquet spark_catalog.default.customer (29) + +- ReusedExchange (34) (1) Scan parquet spark_catalog.default.store_sales @@ -168,110 +164,88 @@ Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, 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))] -(27) ColumnarToRow [codegen id : 1] +(27) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(28) Exchange -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 : 4] +(28) CometHashAggregate 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#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] +(29) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] 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)) - -(32) ColumnarToRow [codegen id : 2] -Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] - -(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=2] - -(34) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#27] -Join type: Inner -Join condition: None +(30) CometFilter +Input [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] +Condition : (isnotnull(c_customer_sk#22) AND isnotnull(c_current_addr_sk#23)) -(35) Project [codegen id : 4] -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] +(31) CometBroadcastExchange +Input [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] +Arguments: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] -(36) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#31, ca_city#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] -ReadSchema: struct +(32) CometBroadcastHashJoin +Left output [5]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#26, amt#27, profit#28] +Right output [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] +Arguments: [ss_customer_sk#1], [c_customer_sk#22], Inner, BuildRight -(37) CometFilter -Input [2]: [ca_address_sk#31, ca_city#32] -Condition : (isnotnull(ca_address_sk#31) AND isnotnull(ca_city#32)) +(33) CometProject +Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#26, amt#27, profit#28, c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] +Arguments: [ss_ticket_number#5, bought_city#26, amt#27, profit#28, c_current_addr_sk#23, c_first_name#24, c_last_name#25], [ss_ticket_number#5, bought_city#26, amt#27, profit#28, c_current_addr_sk#23, c_first_name#24, c_last_name#25] -(38) ColumnarToRow [codegen id : 3] -Input [2]: [ca_address_sk#31, ca_city#32] +(34) ReusedExchange [Reuses operator id: 23] +Output [2]: [ca_address_sk#29, ca_city#30] -(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] +(35) CometBroadcastHashJoin +Left output [7]: [ss_ticket_number#5, bought_city#26, amt#27, profit#28, c_current_addr_sk#23, c_first_name#24, c_last_name#25] +Right output [2]: [ca_address_sk#29, ca_city#30] +Arguments: [c_current_addr_sk#23], [ca_address_sk#29], Inner, NOT (ca_city#30 = bought_city#26), BuildRight -(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) +(36) CometProject +Input [9]: [ss_ticket_number#5, bought_city#26, amt#27, profit#28, c_current_addr_sk#23, c_first_name#24, c_last_name#25, ca_address_sk#29, ca_city#30] +Arguments: [c_last_name#25, c_first_name#24, ca_city#30, bought_city#26, ss_ticket_number#5, amt#27, profit#28], [c_last_name#25, c_first_name#24, ca_city#30, bought_city#26, ss_ticket_number#5, amt#27, profit#28] -(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] +(37) CometTakeOrderedAndProject +Input [7]: [c_last_name#25, c_first_name#24, ca_city#30, bought_city#26, ss_ticket_number#5, amt#27, profit#28] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#25 ASC NULLS FIRST,c_first_name#24 ASC NULLS FIRST,ca_city#30 ASC NULLS FIRST,bought_city#26 ASC NULLS FIRST,ss_ticket_number#5 ASC NULLS FIRST], output=[c_last_name#25,c_first_name#24,ca_city#30,bought_city#26,ss_ticket_number#5,amt#27,profit#28]), [c_last_name#25, c_first_name#24, ca_city#30, bought_city#26, ss_ticket_number#5, amt#27, profit#28], 100, [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, ca_city#30 ASC NULLS FIRST, bought_city#26 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#25, c_first_name#24, ca_city#30, bought_city#26, ss_ticket_number#5, amt#27, profit#28] -(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] +(38) ColumnarToRow [codegen id : 1] +Input [7]: [c_last_name#25, c_first_name#24, ca_city#30, 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 (47) -+- * ColumnarToRow (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan parquet spark_catalog.default.date_dim (43) +BroadcastExchange (43) ++- * ColumnarToRow (42) + +- CometProject (41) + +- CometFilter (40) + +- CometScan parquet spark_catalog.default.date_dim (39) -(43) Scan parquet spark_catalog.default.date_dim +(39) 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 -(44) CometFilter +(40) 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)) -(45) CometProject +(41) CometProject Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(46) ColumnarToRow [codegen id : 1] +(42) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(47) BroadcastExchange +(43) 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/q46/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt index 57defd961..1d28a8ede 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,60 +1,48 @@ -TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] - 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] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,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_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_address_sk,ca_city] - CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,hd_demo_sk] - CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,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_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [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] - 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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_city] - CometScan parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange [hd_demo_sk] #5 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [ca_address_sk,ca_city] #6 - CometFilter [ca_address_sk,ca_city] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_city] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] + CometProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] + CometBroadcastHashJoin [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk,ca_city] + CometProject [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name] + CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,bought_city,amt,profit,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometHashAggregate [ss_ticket_number,ss_customer_sk,bought_city,amt,profit,ss_addr_sk,ca_city,sum,sum,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] + CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,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_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_address_sk,ca_city] + CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,hd_demo_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,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_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [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] + 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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometBroadcastExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometBroadcastExchange [s_store_sk] #4 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_city] + CometScan parquet spark_catalog.default.store [s_store_sk,s_city] + CometBroadcastExchange [hd_demo_sk] #5 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange [ca_address_sk,ca_city] #6 + CometFilter [ca_address_sk,ca_city] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] #7 + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + ReusedExchange [ca_address_sk,ca_city] #6 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 ce5faa952..3fe3003ce 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 @@ -2,18 +2,18 @@ TakeOrderedAndProject (45) +- * Project (44) +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) + :- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) : :- * Project (28) : : +- * Filter (27) : : +- Window (26) : : +- * Filter (25) : : +- Window (24) - : : +- * Sort (23) - : : +- Exchange (22) - : : +- * HashAggregate (21) - : : +- Exchange (20) - : : +- * ColumnarToRow (19) + : : +- * ColumnarToRow (23) + : : +- CometSort (22) + : : +- CometColumnarExchange (21) + : : +- CometHashAggregate (20) + : : +- CometColumnarExchange (19) : : +- CometHashAggregate (18) : : +- CometProject (17) : : +- CometBroadcastHashJoin (16) @@ -32,18 +32,18 @@ TakeOrderedAndProject (45) : : +- CometBroadcastExchange (15) : : +- CometFilter (14) : : +- CometScan parquet spark_catalog.default.store (13) - : +- BroadcastExchange (35) - : +- * Project (34) - : +- Window (33) - : +- * Sort (32) - : +- Exchange (31) - : +- * HashAggregate (30) - : +- ReusedExchange (29) + : +- BroadcastExchange (34) + : +- * Project (33) + : +- Window (32) + : +- * ColumnarToRow (31) + : +- CometSort (30) + : +- ReusedExchange (29) +- BroadcastExchange (42) +- * Project (41) +- Window (40) - +- * Sort (39) - +- ReusedExchange (38) + +- * ColumnarToRow (39) + +- CometSort (38) + +- ReusedExchange (37) (1) Scan parquet spark_catalog.default.item @@ -135,120 +135,113 @@ Input [7]: [i_brand#2, i_category#3, ss_sales_price#6, d_year#10, d_moy#11, s_st 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))] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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#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 : 2] +(20) CometHashAggregate 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))#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#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] +(21) CometColumnarExchange +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, _w0#17] +Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(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 +(22) CometSort +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, _w0#17] +Arguments: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, _w0#17], [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] + +(23) ColumnarToRow [codegen id : 1] +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, _w0#17] (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#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] +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 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, 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#18], [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 : 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] +(25) Filter [codegen id : 2] +Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 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 [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 : 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 : 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#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum#27] - -(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#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 : 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#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 : 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#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 : 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)] +Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, _w0#17, rn#18] +Arguments: [avg(_w0#17) 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#19], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10] + +(27) Filter [codegen id : 7] +Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 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 : 7] +Output [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18] +Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, _w0#17, rn#18, avg_monthly_sales#19] + +(29) ReusedExchange [Reuses operator id: 21] +Output [7]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#16] + +(30) CometSort +Input [7]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#16] +Arguments: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#16], [i_category#20 ASC NULLS FIRST, i_brand#21 ASC NULLS FIRST, s_store_name#22 ASC NULLS FIRST, s_company_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] + +(31) ColumnarToRow [codegen id : 3] +Input [7]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#16] + +(32) Window +Input [7]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#16] +Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#20, i_brand#21, s_store_name#22, s_company_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#26], [i_category#20, i_brand#21, s_store_name#22, s_company_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] + +(33) Project [codegen id : 4] +Output [6]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, sum_sales#16 AS sum_sales#27, rn#26] +Input [8]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#16, rn#26] + +(34) BroadcastExchange +Input [6]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, sum_sales#27, rn#26] +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=3] + +(35) BroadcastHashJoin [codegen id : 7] +Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#18] +Right keys [5]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, (rn#26 + 1)] Join type: Inner Join condition: None -(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] +(36) Project [codegen id : 7] +Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, sum_sales#27] +Input [15]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, i_category#20, i_brand#21, s_store_name#22, s_company_name#23, sum_sales#27, rn#26] + +(37) ReusedExchange [Reuses operator id: 21] +Output [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#16] -(38) ReusedExchange [Reuses operator id: 31] -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] +(38) CometSort +Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#16] +Arguments: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#16], [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_name#30 ASC NULLS FIRST, s_company_name#31 ASC NULLS FIRST, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] -(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 +(39) ColumnarToRow [codegen id : 5] +Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#16] (40) Window -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] +Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#16] +Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#34], [i_category#28, i_brand#29, s_store_name#30, s_company_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] -(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] +(41) Project [codegen id : 6] +Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#16 AS sum_sales#35, rn#34] +Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#16, rn#34] (42) BroadcastExchange -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] +Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#35, rn#34] +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] -(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)] +(43) BroadcastHashJoin [codegen id : 7] +Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#18] +Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#34 - 1)] Join type: Inner Join condition: None -(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] +(44) Project [codegen id : 7] +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#19, sum_sales#16, sum_sales#27 AS psum#36, sum_sales#35 AS nsum#37] +Input [16]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, sum_sales#27, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#35, rn#34] (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#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] +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#19, sum_sales#16, psum#36, nsum#37] +Arguments: 100, [(sum_sales#16 - avg_monthly_sales#19) 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#19, sum_sales#16, psum#36, nsum#37] ===== Subqueries ===== @@ -275,6 +268,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=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] 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 b6e5c469b..91c2cd65d 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 (13) + WholeStageCodegen (7) 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,67 +8,61 @@ 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 (4) + WholeStageCodegen (2) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (3) - Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum,ss_sales_price] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometFilter [i_item_sk,i_brand,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #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] + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 + CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum,sum(UnscaledValue(ss_sales_price))] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum,ss_sales_price] + CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #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 #7 - WholeStageCodegen (8) + WholeStageCodegen (4) 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 (7) - Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + WholeStageCodegen (3) + ColumnarToRow InputAdapter - 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 + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #1 InputAdapter - BroadcastExchange #9 - WholeStageCodegen (12) + BroadcastExchange #8 + WholeStageCodegen (6) 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 (11) - Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + WholeStageCodegen (5) + ColumnarToRow InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #1 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 6b4d61a5e..3393afedf 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,7 +1,7 @@ == Physical Plan == -* HashAggregate (28) -+- Exchange (27) - +- * ColumnarToRow (26) +* ColumnarToRow (28) ++- CometHashAggregate (27) + +- CometColumnarExchange (26) +- CometHashAggregate (25) +- CometProject (24) +- CometBroadcastHashJoin (23) @@ -150,19 +150,17 @@ Input [1]: [ss_quantity#4] Keys: [] Functions [1]: [partial_sum(ss_quantity#4)] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarExchange Input [1]: [sum#18] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(27) Exchange -Input [1]: [sum#18] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] - -(28) HashAggregate [codegen id : 2] +(27) CometHashAggregate Input [1]: [sum#18] 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] + +(28) ColumnarToRow [codegen id : 1] +Input [1]: [sum(ss_quantity)#19] ===== Subqueries ===== 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 5f628dbae..8bb0cede6 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,40 +1,38 @@ -WholeStageCodegen (2) - HashAggregate [sum] [sum(ss_quantity),sum(ss_quantity),sum] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [sum,ss_quantity] - CometProject [ss_quantity] - CometBroadcastHashJoin [ss_quantity,ss_sold_date_sk,d_date_sk] - CometProject [ss_quantity,ss_sold_date_sk] - CometBroadcastHashJoin [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk,ca_address_sk,ca_state] - CometProject [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,cd_demo_sk,cd_marital_status,cd_education_status] - CometProject [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,s_store_sk] - CometFilter [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] - 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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk] #3 - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #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 [ca_address_sk,ca_state] #5 - CometProject [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometHashAggregate [sum(ss_quantity),sum,sum(ss_quantity)] + CometColumnarExchange #1 + CometHashAggregate [sum,ss_quantity] + CometProject [ss_quantity] + CometBroadcastHashJoin [ss_quantity,ss_sold_date_sk,d_date_sk] + CometProject [ss_quantity,ss_sold_date_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk,ca_address_sk,ca_state] + CometProject [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk] + CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,cd_demo_sk,cd_marital_status,cd_education_status] + CometProject [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] + CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,s_store_sk] + CometFilter [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] + 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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [s_store_sk] #3 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #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 [ca_address_sk,ca_state] #5 + CometProject [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + 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 7df26543a..4b0d5aed0 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,81 +1,83 @@ == Physical Plan == -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) +* ColumnarToRow (79) ++- CometTakeOrderedAndProject (78) + +- CometHashAggregate (77) + +- CometColumnarExchange (76) + +- RowToColumnar (75) + +- * HashAggregate (74) + +- Union (73) + :- * Project (26) + : +- * Filter (25) + : +- Window (24) + : +- * Sort (23) + : +- Window (22) + : +- * ColumnarToRow (21) + : +- CometSort (20) + : +- CometColumnarExchange (19) + : +- CometHashAggregate (18) + : +- CometColumnarExchange (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) + : +- * ColumnarToRow (44) + : +- CometSort (43) + : +- CometColumnarExchange (42) + : +- CometHashAggregate (41) + : +- CometColumnarExchange (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) + +- * ColumnarToRow (67) + +- CometSort (66) + +- CometColumnarExchange (65) + +- CometHashAggregate (64) + +- CometColumnarExchange (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 @@ -155,312 +157,310 @@ Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, w 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))] -(17) ColumnarToRow [codegen id : 1] +(17) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(19) HashAggregate [codegen id : 2] +(18) CometHashAggregate 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))#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] -(20) Exchange -Input [3]: [item#26, return_ratio#27, currency_ratio#28] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] +(19) CometColumnarExchange +Input [3]: [item#22, return_ratio#23, currency_ratio#24] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(20) CometSort +Input [3]: [item#22, return_ratio#23, currency_ratio#24] +Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] -(21) Sort [codegen id : 3] -Input [3]: [item#26, return_ratio#27, currency_ratio#28] -Arguments: [return_ratio#27 ASC NULLS FIRST], false, 0 +(21) ColumnarToRow [codegen id : 1] +Input [3]: [item#22, return_ratio#23, currency_ratio#24] (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] +Input [3]: [item#22, return_ratio#23, currency_ratio#24] +Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] -(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 +(23) Sort [codegen id : 2] +Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] +Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 (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] +Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] +Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] -(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)) +(25) Filter [codegen id : 3] +Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] +Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) -(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] +(26) Project [codegen id : 3] +Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] (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] +Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#37), dynamicpruningexpression(cs_sold_date_sk#37 IN dynamicpruning#38)] +PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_sold_date_sk#33 IN dynamicpruning#34)] 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 (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)) +Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] +Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) (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] +Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] +Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] (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] +Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] +Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] (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] +Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] 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 (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)) +Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] +Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) (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] +Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] +Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] (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 +Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] +Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] +Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], 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] +Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] +Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] (36) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#44] +Output [1]: [d_date_sk#40] (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 +Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] +Right output [1]: [d_date_sk#40] +Arguments: [cs_sold_date_sk#33], [d_date_sk#40], 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] +Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] +Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] (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))] +Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] +Keys [1]: [cs_item_sk#28] +Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 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] +(40) CometColumnarExchange +Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] +Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(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] +(41) CometHashAggregate +Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] +Keys [1]: [cs_item_sk#28] +Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(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] +(42) CometColumnarExchange +Input [3]: [item#47, return_ratio#48, currency_ratio#49] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(43) Exchange -Input [3]: [item#55, return_ratio#56, currency_ratio#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +(43) CometSort +Input [3]: [item#47, return_ratio#48, currency_ratio#49] +Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] -(44) Sort [codegen id : 8] -Input [3]: [item#55, return_ratio#56, currency_ratio#57] -Arguments: [return_ratio#56 ASC NULLS FIRST], false, 0 +(44) ColumnarToRow [codegen id : 4] +Input [3]: [item#47, return_ratio#48, currency_ratio#49] (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] +Input [3]: [item#47, return_ratio#48, currency_ratio#49] +Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] -(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 +(46) Sort [codegen id : 5] +Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] +Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 (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] +Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] +Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] -(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)) +(48) Filter [codegen id : 6] +Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] +Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) -(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] +(49) Project [codegen id : 6] +Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] +Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] (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] +Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#66), dynamicpruningexpression(ss_sold_date_sk#66 IN dynamicpruning#67)] +PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] 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 (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)) +Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] +Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) (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] +Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] +Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] (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] +Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] +Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] (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] +Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] 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 (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)) +Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] +Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) (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] +Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] +Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] (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 +Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] +Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] +Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], 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] +Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] +Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] (59) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#73] +Output [1]: [d_date_sk#65] (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 +Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] +Right output [1]: [d_date_sk#65] +Arguments: [ss_sold_date_sk#58], [d_date_sk#65], 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] +Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] +Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] (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))] +Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] +Keys [1]: [ss_item_sk#53] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 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] +(63) CometColumnarExchange +Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] +Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(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] +(64) CometHashAggregate +Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] +Keys [1]: [ss_item_sk#53] +Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(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] +(65) CometColumnarExchange +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(66) Exchange -Input [3]: [item#84, return_ratio#85, currency_ratio#86] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +(66) CometSort +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] -(67) Sort [codegen id : 13] -Input [3]: [item#84, return_ratio#85, currency_ratio#86] -Arguments: [return_ratio#85 ASC NULLS FIRST], false, 0 +(67) ColumnarToRow [codegen id : 7] +Input [3]: [item#72, return_ratio#73, currency_ratio#74] (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] +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] -(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 +(69) Sort [codegen id : 8] +Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] +Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 (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] +Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] +Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] -(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)) +(71) Filter [codegen id : 9] +Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] +Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) -(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] +(72) Project [codegen id : 9] +Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] +Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] (73) Union -(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] +(74) HashAggregate [codegen id : 10] +Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(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] +(75) RowToColumnar +Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(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] +(76) CometColumnarExchange +Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(77) CometHashAggregate +Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] -(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] +(78) CometTakeOrderedAndProject +Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] + +(79) ColumnarToRow [codegen id : 11] +Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (82) -+- * ColumnarToRow (81) - +- CometProject (80) - +- CometFilter (79) - +- CometScan parquet spark_catalog.default.date_dim (78) +BroadcastExchange (84) ++- * ColumnarToRow (83) + +- CometProject (82) + +- CometFilter (81) + +- CometScan parquet spark_catalog.default.date_dim (80) -(78) Scan parquet spark_catalog.default.date_dim +(80) 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 -(79) CometFilter +(81) 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)) -(80) CometProject +(82) CometProject Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(81) ColumnarToRow [codegen id : 1] +(83) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(82) BroadcastExchange +(84) 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 = 27 Hosting Expression = cs_sold_date_sk#37 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#66 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 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 0e6b65b06..0d02a9c39 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,121 +1,111 @@ -TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] - WholeStageCodegen (17) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Exchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (16) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (5) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (4) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (3) - Sort [return_ratio] - InputAdapter - Exchange #2 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,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_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - 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 (9) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (8) - Sort [return_ratio] - InputAdapter - 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] #8 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometHashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,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_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_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] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - 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 (14) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (13) - Sort [return_ratio] - InputAdapter - 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] #11 - WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometHashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,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_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_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] - ReusedSubquery [d_date_sk] #1 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - ReusedExchange [d_date_sk] #6 +WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] + CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] + CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] + InputAdapter + Union + WholeStageCodegen (3) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (2) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #2 + CometHashAggregate [item,return_ratio,currency_ratio,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))] + CometColumnarExchange [ws_item_sk] #3 + CometHashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,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_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + WholeStageCodegen (6) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (5) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #7 + CometHashAggregate [item,return_ratio,currency_ratio,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))] + CometColumnarExchange [cs_item_sk] #8 + CometHashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,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_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_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] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (9) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (8) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #10 + CometHashAggregate [item,return_ratio,currency_ratio,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))] + CometColumnarExchange [ss_item_sk] #11 + CometHashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,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_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_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] + ReusedSubquery [d_date_sk] #1 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + 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 07bd5f03a..ed1f8feda 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 @@ -1,74 +1,76 @@ == Physical Plan == -TakeOrderedAndProject (70) -+- * HashAggregate (69) - +- Exchange (68) - +- * HashAggregate (67) - +- * Expand (66) - +- Union (65) - :- * 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) - +- * 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) +TakeOrderedAndProject (72) ++- * HashAggregate (71) + +- * ColumnarToRow (70) + +- CometColumnarExchange (69) + +- RowToColumnar (68) + +- * HashAggregate (67) + +- * Expand (66) + +- Union (65) + :- * HashAggregate (22) + : +- * ColumnarToRow (21) + : +- CometColumnarExchange (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) + : +- * ColumnarToRow (40) + : +- CometColumnarExchange (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) + +- * ColumnarToRow (63) + +- CometColumnarExchange (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) (1) Scan parquet spark_catalog.default.store_sales @@ -164,14 +166,14 @@ 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))] -(20) ColumnarToRow [codegen id : 1] +(20) CometColumnarExchange Input [5]: [s_store_id#25, sum#26, sum#27, sum#28, sum#29] +Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(21) Exchange +(21) ColumnarToRow [codegen id : 1] 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] -(22) HashAggregate [codegen id : 2] +(22) HashAggregate [codegen id : 1] 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))] @@ -255,14 +257,14 @@ Input [5]: [sales_price#46, profit#47, return_amt#48, net_loss#49, cp_catalog_pa 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] +(39) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=2] -(40) Exchange +(40) ColumnarToRow [codegen id : 2] 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] +(41) HashAggregate [codegen id : 2] 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))] @@ -365,14 +367,14 @@ Input [5]: [sales_price#83, profit#84, return_amt#85, net_loss#86, web_site_id#1 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) ColumnarToRow [codegen id : 5] +(62) CometColumnarExchange Input [5]: [web_site_id#104, sum#105, sum#106, sum#107, sum#108] +Arguments: hashpartitioning(web_site_id#104, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(63) Exchange +(63) ColumnarToRow [codegen id : 3] 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 : 6] +(64) HashAggregate [codegen id : 3] 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))] @@ -381,61 +383,67 @@ Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#83))#109,17,2) AS sales# (65) Union -(66) Expand [codegen id : 7] +(66) Expand [codegen id : 4] 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 : 7] +(67) HashAggregate [codegen id : 4] 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 +(68) RowToColumnar 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 : 8] +(69) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=4] + +(70) ColumnarToRow [codegen id : 5] +Input [9]: [channel#118, id#119, spark_grouping_id#120, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] + +(71) HashAggregate [codegen id : 5] 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 +(72) TakeOrderedAndProject 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 ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (75) -+- * ColumnarToRow (74) - +- CometProject (73) - +- CometFilter (72) - +- CometScan parquet spark_catalog.default.date_dim (71) +BroadcastExchange (77) ++- * ColumnarToRow (76) + +- CometProject (75) + +- CometFilter (74) + +- CometScan parquet spark_catalog.default.date_dim (73) -(71) Scan parquet spark_catalog.default.date_dim +(73) 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 -(72) CometFilter +(74) 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)) -(73) CometProject +(75) CometProject Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(74) ColumnarToRow [codegen id : 1] +(76) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(75) BroadcastExchange +(77) BroadcastExchange Input [1]: [d_date_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=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 537cba446..7b4b411a4 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,20 +1,20 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (8) + WholeStageCodegen (5) 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 (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 (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 (1) - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometColumnarExchange [channel,id,spark_grouping_id] #1 + RowToColumnar + WholeStageCodegen (4) + 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 (1) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [s_store_id] #2 CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,s_store_id] CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] @@ -43,13 +43,11 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometBroadcastExchange [s_store_sk,s_store_id] #5 CometFilter [s_store_sk,s_store_id] 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] #6 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter + WholeStageCodegen (2) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [cp_catalog_page_id] #6 CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] @@ -68,13 +66,11 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #7 CometFilter [cp_catalog_page_sk,cp_catalog_page_id] 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] #8 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter + WholeStageCodegen (3) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [web_site_id] #8 CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,web_site_id] CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,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 c1c7074be..1e9d7743c 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 @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (28) -+- * HashAggregate (27) - +- Exchange (26) - +- * ColumnarToRow (25) +* ColumnarToRow (28) ++- CometTakeOrderedAndProject (27) + +- CometHashAggregate (26) + +- CometColumnarExchange (25) +- CometHashAggregate (24) +- CometProject (23) +- CometBroadcastHashJoin (22) @@ -147,23 +147,21 @@ Input [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_compan 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)] -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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#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 : 2] +(26) CometHashAggregate 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)#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 #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] +(27) CometTakeOrderedAndProject +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 #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[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], output=[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 #31,31 - 60 days #32,61 - 90 days #33,91 - 120 days #34,>120 days #35]), [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 #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35], 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 #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] + +(28) 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, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] ===== Subqueries ===== 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 58adabcec..075786f9b 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,40 +1,38 @@ -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 (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 (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,sum,sum,sum,sum,sum,sr_returned_date_sk,ss_sold_date_sk] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [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 ] + 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,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,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)] + CometColumnarExchange [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 + 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,sum,sum,sum,sum,sum,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 [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,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,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,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,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,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,ss_sold_date_sk,sr_returned_date_sk,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] - CometProject [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] #2 - CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [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] #4 - CometFilter [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] - 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 [d_date_sk] #5 - CometFilter [d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastHashJoin [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk,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] + CometProject [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] #2 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_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_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [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] #4 + CometFilter [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] + 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 [d_date_sk] #5 + CometFilter [d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + 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 bc106394c..3bd86214d 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,44 +1,47 @@ == Physical Plan == -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) +* ColumnarToRow (43) ++- CometTakeOrderedAndProject (42) + +- CometFilter (41) + +- CometWindowExec (40) + +- CometSort (39) + +- CometColumnarExchange (38) + +- CometProject (37) + +- CometSortMergeJoin (36) + :- CometSort (19) + : +- CometColumnarExchange (18) + : +- RowToColumnar (17) + : +- * Project (16) + : +- Window (15) + : +- * ColumnarToRow (14) + : +- CometSort (13) + : +- CometColumnarExchange (12) + : +- CometHashAggregate (11) + : +- CometColumnarExchange (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) + +- CometSort (35) + +- CometColumnarExchange (34) + +- RowToColumnar (33) + +- * Project (32) + +- Window (31) + +- * ColumnarToRow (30) + +- CometSort (29) + +- CometColumnarExchange (28) + +- CometHashAggregate (27) + +- CometColumnarExchange (26) + +- CometHashAggregate (25) + +- CometProject (24) + +- CometBroadcastHashJoin (23) + :- CometFilter (21) + : +- CometScan parquet spark_catalog.default.store_sales (20) + +- ReusedExchange (22) (1) Scan parquet spark_catalog.default.web_sales @@ -86,173 +89,177 @@ 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))] -(10) ColumnarToRow [codegen id : 1] +(10) CometColumnarExchange Input [3]: [ws_item_sk#1, d_date#6, sum#8] +Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(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] - -(12) HashAggregate [codegen id : 2] +(11) CometHashAggregate 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] -(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] +(12) CometColumnarExchange +Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(13) CometSort +Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] +Arguments: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(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 +(14) ColumnarToRow [codegen id : 1] +Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] (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] +Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] +Arguments: [sum(_w0#10) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(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] +(16) Project [codegen id : 2] +Output [3]: [item_sk#9, d_date#6, cume_sales#11] +Input [5]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1, cume_sales#11] -(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] +(17) RowToColumnar +Input [3]: [item_sk#9, d_date#6, cume_sales#11] -(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 +(18) CometColumnarExchange +Input [3]: [item_sk#9, d_date#6, cume_sales#11] +Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(19) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] +(19) CometSort +Input [3]: [item_sk#9, d_date#6, cume_sales#11] +Arguments: [item_sk#9, d_date#6, cume_sales#11], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] + +(20) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] 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#14), dynamicpruningexpression(ss_sold_date_sk#14 IN dynamicpruning#15)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(20) CometFilter -Input [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] -Condition : isnotnull(ss_item_sk#13) +(21) CometFilter +Input [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] +Condition : isnotnull(ss_item_sk#12) + +(22) ReusedExchange [Reuses operator id: 6] +Output [2]: [d_date_sk#16, d_date#17] + +(23) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] +Right output [2]: [d_date_sk#16, d_date#17] +Arguments: [ss_sold_date_sk#14], [d_date_sk#16], Inner, BuildRight -(21) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#17, d_date#18] +(24) CometProject +Input [5]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14, d_date_sk#16, d_date#17] +Arguments: [ss_item_sk#12, ss_sales_price#13, d_date#17], [ss_item_sk#12, ss_sales_price#13, d_date#17] -(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 +(25) CometHashAggregate +Input [3]: [ss_item_sk#12, ss_sales_price#13, d_date#17] +Keys [2]: [ss_item_sk#12, d_date#17] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#13))] -(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] +(26) CometColumnarExchange +Input [3]: [ss_item_sk#12, d_date#17, sum#18] +Arguments: hashpartitioning(ss_item_sk#12, d_date#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(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))] +(27) CometHashAggregate +Input [3]: [ss_item_sk#12, d_date#17, sum#18] +Keys [2]: [ss_item_sk#12, d_date#17] +Functions [1]: [sum(UnscaledValue(ss_sales_price#13))] -(25) ColumnarToRow [codegen id : 6] -Input [3]: [ss_item_sk#13, d_date#18, sum#19] +(28) CometColumnarExchange +Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] +Arguments: hashpartitioning(ss_item_sk#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(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] +(29) CometSort +Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] +Arguments: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12], [ss_item_sk#12 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] -(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))#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] +(30) ColumnarToRow [codegen id : 3] +Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] -(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] +(31) Window +Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] +Arguments: [sum(_w0#20) windowspecdefinition(ss_item_sk#12, d_date#17 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#21], [ss_item_sk#12], [d_date#17 ASC NULLS FIRST] -(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 +(32) Project [codegen id : 4] +Output [3]: [item_sk#19, d_date#17, cume_sales#21] +Input [5]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12, cume_sales#21] -(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] +(33) RowToColumnar +Input [3]: [item_sk#19, d_date#17, cume_sales#21] -(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] +(34) CometColumnarExchange +Input [3]: [item_sk#19, d_date#17, cume_sales#21] +Arguments: hashpartitioning(item_sk#19, d_date#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [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] +(35) CometSort +Input [3]: [item_sk#19, d_date#17, cume_sales#21] +Arguments: [item_sk#19, d_date#17, cume_sales#21], [item_sk#19 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] -(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 +(36) CometSortMergeJoin +Left output [3]: [item_sk#9, d_date#6, cume_sales#11] +Right output [3]: [item_sk#19, d_date#17, cume_sales#21] +Arguments: [item_sk#9, d_date#6], [item_sk#19, d_date#17], FullOuter -(34) SortMergeJoin [codegen id : 11] -Left keys [2]: [item_sk#10, d_date#6] -Right keys [2]: [item_sk#21, d_date#18] -Join type: FullOuter -Join condition: None +(37) CometProject +Input [6]: [item_sk#9, d_date#6, cume_sales#11, item_sk#19, d_date#17, cume_sales#21] +Arguments: [item_sk#22, d_date#23, web_sales#24, store_sales#25], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#19 END AS item_sk#22, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#17 END AS d_date#23, cume_sales#11 AS web_sales#24, cume_sales#21 AS store_sales#25] -(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] +(38) CometColumnarExchange +Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] +Arguments: hashpartitioning(item_sk#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [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] +(39) CometSort +Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] +Arguments: [item_sk#22, d_date#23, web_sales#24, store_sales#25], [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST] -(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 +(40) CometWindowExec +Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] +Arguments: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27], [max(web_sales#24) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#26, max(store_sales#25) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#27], [item_sk#22], [d_date#23 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] +(41) CometFilter +Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] +Condition : ((isnotnull(web_cumulative#26) AND isnotnull(store_cumulative#27)) AND (web_cumulative#26 > store_cumulative#27)) -(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)) +(42) CometTakeOrderedAndProject +Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_sk#22 ASC NULLS FIRST,d_date#23 ASC NULLS FIRST], output=[item_sk#22,d_date#23,web_sales#24,store_sales#25,web_cumulative#26,store_cumulative#27]), [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27], 100, [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST], [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] -(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] +(43) ColumnarToRow [codegen id : 5] +Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -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 +(44) 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 -(42) CometFilter +(45) 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)) -(43) CometProject +(46) 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] -(44) ColumnarToRow [codegen id : 1] +(47) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(45) BroadcastExchange +(48) 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 = 19 Hosting Expression = ss_sold_date_sk#15 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#14 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 988297f02..0e5a8e3d7 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,76 +1,62 @@ -TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (13) - Filter [web_cumulative,store_cumulative] - InputAdapter - Window [web_sales,item_sk,d_date,store_sales] - WholeStageCodegen (12) - Sort [item_sk,d_date] - InputAdapter - Exchange [item_sk] #1 - 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 (5) - Sort [item_sk,d_date] +WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + CometFilter [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + CometWindowExec [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + CometSort [item_sk,d_date,web_sales,store_sales] + CometColumnarExchange [item_sk] #1 + CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] + CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #2 + RowToColumnar + WholeStageCodegen (2) + Project [item_sk,d_date,cume_sales] InputAdapter - Exchange [item_sk,d_date] #2 - WholeStageCodegen (4) - Project [item_sk,d_date,cume_sales] + Window [_w0,ws_item_sk,d_date] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Window [_w0,ws_item_sk,d_date] - WholeStageCodegen (3) - Sort [ws_item_sk,d_date] - InputAdapter - Exchange [ws_item_sk] #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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ws_item_sk,d_date,sum,ws_sales_price] - CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #6 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - WholeStageCodegen (10) - Sort [item_sk,d_date] + CometSort [item_sk,d_date,_w0,ws_item_sk] + CometColumnarExchange [ws_item_sk] #3 + CometHashAggregate [item_sk,d_date,_w0,ws_item_sk,sum,sum(UnscaledValue(ws_sales_price))] + CometColumnarExchange [ws_item_sk,d_date] #4 + CometHashAggregate [ws_item_sk,d_date,sum,ws_sales_price] + CometProject [ws_item_sk,ws_sales_price,d_date] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange [d_date_sk,d_date] #6 + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #7 + RowToColumnar + WholeStageCodegen (4) + Project [item_sk,d_date,cume_sales] InputAdapter - Exchange [item_sk,d_date] #7 - WholeStageCodegen (9) - Project [item_sk,d_date,cume_sales] + Window [_w0,ss_item_sk,d_date] + WholeStageCodegen (3) + ColumnarToRow InputAdapter - Window [_w0,ss_item_sk,d_date] - WholeStageCodegen (8) - Sort [ss_item_sk,d_date] - InputAdapter - 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] #9 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometHashAggregate [ss_item_sk,d_date,sum,ss_sales_price] - CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_date] #6 + CometSort [item_sk,d_date,_w0,ss_item_sk] + CometColumnarExchange [ss_item_sk] #8 + CometHashAggregate [item_sk,d_date,_w0,ss_item_sk,sum,sum(UnscaledValue(ss_sales_price))] + CometColumnarExchange [ss_item_sk,d_date] #9 + CometHashAggregate [ss_item_sk,d_date,sum,ss_sales_price] + CometProject [ss_item_sk,ss_sales_price,d_date] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + 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 65c777125..f8beff744 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,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (19) -+- * HashAggregate (18) - +- Exchange (17) - +- * ColumnarToRow (16) +* ColumnarToRow (19) ++- CometTakeOrderedAndProject (18) + +- CometHashAggregate (17) + +- CometColumnarExchange (16) +- CometHashAggregate (15) +- CometProject (14) +- CometBroadcastHashJoin (13) @@ -93,21 +93,19 @@ 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))] -(16) ColumnarToRow [codegen id : 1] +(16) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(18) HashAggregate [codegen id : 2] +(17) CometHashAggregate 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))#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] -(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] +(18) CometTakeOrderedAndProject +Input [4]: [d_year#2, brand_id#12, brand#13, ext_price#14] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,ext_price#14 DESC NULLS LAST,brand_id#12 ASC NULLS FIRST], output=[d_year#2,brand_id#12,brand#13,ext_price#14]), [d_year#2, brand_id#12, brand#13, ext_price#14], 100, [d_year#2 ASC NULLS FIRST, ext_price#14 DESC NULLS LAST, brand_id#12 ASC NULLS FIRST], [d_year#2, brand_id#12, brand#13, ext_price#14] + +(19) ColumnarToRow [codegen id : 1] +Input [4]: [d_year#2, brand_id#12, brand#13, ext_price#14] 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 a0932c7cc..99f3baf91 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,23 +1,21 @@ -TakeOrderedAndProject [d_year,ext_price,brand_id,brand] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [d_year,i_brand,i_brand_id,sum,ss_ext_sales_price] - CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] - CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [d_year,brand_id,brand,ext_price] + CometHashAggregate [d_year,brand_id,brand,ext_price,i_brand,i_brand_id,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange [d_year,i_brand,i_brand_id] #1 + CometHashAggregate [d_year,i_brand,i_brand_id,sum,ss_ext_sales_price] + CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] + CometProject [d_year,ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometProject [d_date_sk,d_year] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 + CometProject [i_item_sk,i_brand_id,i_brand] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] + 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 76d80c043..71c84c040 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 @@ -3,11 +3,11 @@ TakeOrderedAndProject (29) +- * Project (28) +- * Filter (27) +- Window (26) - +- * Sort (25) - +- Exchange (24) - +- * HashAggregate (23) - +- Exchange (22) - +- * ColumnarToRow (21) + +- * ColumnarToRow (25) + +- CometSort (24) + +- CometColumnarExchange (23) + +- CometHashAggregate (22) + +- CometColumnarExchange (21) +- CometHashAggregate (20) +- CometProject (19) +- CometBroadcastHashJoin (18) @@ -127,43 +127,41 @@ 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))] -(21) ColumnarToRow [codegen id : 1] +(21) CometColumnarExchange Input [3]: [i_manufact_id#5, d_qoy#17, sum#19] +Arguments: hashpartitioning(i_manufact_id#5, d_qoy#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(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] - -(23) HashAggregate [codegen id : 2] +(22) CometHashAggregate 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] -(24) Exchange -Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] -Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(23) CometColumnarExchange +Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] +Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(24) CometSort +Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] +Arguments: [i_manufact_id#5, sum_sales#20, _w0#21], [i_manufact_id#5 ASC NULLS FIRST] -(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) ColumnarToRow [codegen id : 1] +Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] (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] +Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] +Arguments: [avg(_w0#21) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#22], [i_manufact_id#5] -(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) Filter [codegen id : 2] +Input [4]: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22] +Condition : CASE WHEN (avg_quarterly_sales#22 > 0.000000) THEN ((abs((sum_sales#20 - avg_quarterly_sales#22)) / avg_quarterly_sales#22) > 0.1000000000000000) ELSE false END -(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) Project [codegen id : 2] +Output [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] +Input [4]: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22] (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] +Input [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] +Arguments: 100, [avg_quarterly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] ===== Subqueries ===== 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 b2199fb34..a0d09de86 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,45 +1,41 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] - WholeStageCodegen (4) + WholeStageCodegen (2) Project [i_manufact_id,sum_sales,avg_quarterly_sales] Filter [avg_quarterly_sales,sum_sales] InputAdapter Window [_w0,i_manufact_id] - WholeStageCodegen (3) - Sort [i_manufact_id] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [i_manufact_id] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_manufact_id,d_qoy,sum,ss_sales_price] - CometProject [i_manufact_id,ss_sales_price,d_qoy] - CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] - CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] - CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] - CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_item_sk,i_manufact_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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_date_sk,d_month_seq,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [d_date_sk,d_qoy] #5 - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] + CometSort [i_manufact_id,sum_sales,_w0] + CometColumnarExchange [i_manufact_id] #1 + CometHashAggregate [i_manufact_id,sum_sales,_w0,d_qoy,sum,sum(UnscaledValue(ss_sales_price))] + CometColumnarExchange [i_manufact_id,d_qoy] #2 + CometHashAggregate [i_manufact_id,d_qoy,sum,ss_sales_price] + CometProject [i_manufact_id,ss_sales_price,d_qoy] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] + CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] + CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_item_sk,i_manufact_id] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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_date_sk,d_month_seq,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometBroadcastExchange [d_date_sk,d_qoy] #5 + CometProject [d_date_sk,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometBroadcastExchange [s_store_sk] #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 9eca5ceea..cf1531550 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,60 +1,60 @@ == Physical Plan == -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) +* ColumnarToRow (56) ++- CometTakeOrderedAndProject (55) + +- CometHashAggregate (54) + +- CometColumnarExchange (53) + +- CometHashAggregate (52) + +- CometHashAggregate (51) + +- CometColumnarExchange (50) + +- CometHashAggregate (49) + +- CometProject (48) + +- CometBroadcastHashJoin (47) + :- CometProject (42) + : +- CometBroadcastHashJoin (41) + : :- CometProject (37) + : : +- CometBroadcastHashJoin (36) + : : :- CometProject (32) + : : : +- CometBroadcastHashJoin (31) + : : : :- CometHashAggregate (27) + : : : : +- CometColumnarExchange (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) + : : : +- CometBroadcastExchange (30) + : : : +- CometFilter (29) + : : : +- CometScan parquet spark_catalog.default.store_sales (28) + : : +- CometBroadcastExchange (35) + : : +- CometFilter (34) + : : +- CometScan parquet spark_catalog.default.customer_address (33) + : +- CometBroadcastExchange (40) + : +- CometFilter (39) + : +- CometScan parquet spark_catalog.default.store (38) + +- CometBroadcastExchange (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) (1) Scan parquet spark_catalog.default.catalog_sales @@ -178,21 +178,16 @@ Input [2]: [c_customer_sk#20, c_current_addr_sk#21] Keys [2]: [c_customer_sk#20, c_current_addr_sk#21] Functions: [] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(27) 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#20, c_current_addr_sk#21] -(29) Scan parquet spark_catalog.default.store_sales +(28) 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 [] @@ -200,135 +195,133 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#24), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(30) CometFilter +(29) CometFilter Input [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] Condition : isnotnull(ss_customer_sk#22) -(31) ColumnarToRow [codegen id : 2] -Input [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] - -(32) BroadcastExchange +(30) CometBroadcastExchange 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] +Arguments: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -(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 +(31) CometBroadcastHashJoin +Left output [2]: [c_customer_sk#20, c_current_addr_sk#21] +Right output [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] +Arguments: [c_customer_sk#20], [ss_customer_sk#22], Inner, BuildRight -(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] +(32) CometProject 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] +Arguments: [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24], [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24] -(35) Scan parquet spark_catalog.default.customer_address +(33) 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 -(36) CometFilter +(34) 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)) -(37) ColumnarToRow [codegen id : 3] +(35) CometBroadcastExchange Input [3]: [ca_address_sk#26, ca_county#27, ca_state#28] +Arguments: [ca_address_sk#26, ca_county#27, ca_state#28] -(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] - -(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 +(36) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24] +Right output [3]: [ca_address_sk#26, ca_county#27, ca_state#28] +Arguments: [c_current_addr_sk#21], [ca_address_sk#26], Inner, BuildRight -(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] +(37) CometProject 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] +Arguments: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#28], [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#28] -(41) Scan parquet spark_catalog.default.store +(38) 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 -(42) CometFilter +(39) CometFilter Input [2]: [s_county#29, s_state#30] Condition : (isnotnull(s_county#29) AND isnotnull(s_state#30)) -(43) ColumnarToRow [codegen id : 4] -Input [2]: [s_county#29, s_state#30] - -(44) BroadcastExchange +(40) CometBroadcastExchange Input [2]: [s_county#29, s_state#30] -Arguments: HashedRelationBroadcastMode(List(input[0, string, false], input[1, string, false]),false), [plan_id=4] +Arguments: [s_county#29, s_state#30] -(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 +(41) CometBroadcastHashJoin +Left output [5]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#28] +Right output [2]: [s_county#29, s_state#30] +Arguments: [ca_county#27, ca_state#28], [s_county#29, s_state#30], Inner, BuildRight -(46) Project [codegen id : 6] -Output [3]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24] +(42) CometProject 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] +Arguments: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24], [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24] + +(43) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#31, d_month_seq#32] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] +ReadSchema: struct + +(44) CometFilter +Input [2]: [d_date_sk#31, d_month_seq#32] +Condition : (((isnotnull(d_month_seq#32) AND (d_month_seq#32 >= ReusedSubquery Subquery scalar-subquery#33, [id=#34])) AND (d_month_seq#32 <= ReusedSubquery Subquery scalar-subquery#35, [id=#36])) AND isnotnull(d_date_sk#31)) + +(45) CometProject +Input [2]: [d_date_sk#31, d_month_seq#32] +Arguments: [d_date_sk#31], [d_date_sk#31] -(47) ReusedExchange [Reuses operator id: 66] -Output [1]: [d_date_sk#31] +(46) CometBroadcastExchange +Input [1]: [d_date_sk#31] +Arguments: [d_date_sk#31] -(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 +(47) CometBroadcastHashJoin +Left output [3]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24] +Right output [1]: [d_date_sk#31] +Arguments: [ss_sold_date_sk#24], [d_date_sk#31], Inner, BuildRight -(49) Project [codegen id : 6] -Output [2]: [c_customer_sk#20, ss_ext_sales_price#23] +(48) CometProject Input [4]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, d_date_sk#31] +Arguments: [c_customer_sk#20, ss_ext_sales_price#23], [c_customer_sk#20, ss_ext_sales_price#23] -(50) HashAggregate [codegen id : 6] +(49) CometHashAggregate 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] +(50) CometColumnarExchange +Input [2]: [c_customer_sk#20, sum#37] +Arguments: hashpartitioning(c_customer_sk#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(52) HashAggregate [codegen id : 7] -Input [2]: [c_customer_sk#20, sum#33] +(51) CometHashAggregate +Input [2]: [c_customer_sk#20, sum#37] 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] +(52) CometHashAggregate +Input [1]: [segment#38] +Keys [1]: [segment#38] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#36] -Results [2]: [segment#35, count#37] -(54) Exchange -Input [2]: [segment#35, count#37] -Arguments: hashpartitioning(segment#35, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(53) CometColumnarExchange +Input [2]: [segment#38, count#39] +Arguments: hashpartitioning(segment#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(55) HashAggregate [codegen id : 8] -Input [2]: [segment#35, count#37] -Keys [1]: [segment#35] +(54) CometHashAggregate +Input [2]: [segment#38, count#39] +Keys [1]: [segment#38] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#38] -Results [3]: [segment#35, count(1)#38 AS num_customers#39, (segment#35 * 50) AS segment_base#40] -(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] +(55) CometTakeOrderedAndProject +Input [3]: [segment#38, num_customers#40, segment_base#41] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[segment#38 ASC NULLS FIRST,num_customers#40 ASC NULLS FIRST], output=[segment#38,num_customers#40,segment_base#41]), [segment#38, num_customers#40, segment_base#41], 100, [segment#38 ASC NULLS FIRST, num_customers#40 ASC NULLS FIRST], [segment#38, num_customers#40, segment_base#41] + +(56) ColumnarToRow [codegen id : 1] +Input [3]: [segment#38, num_customers#40, segment_base#41] ===== Subqueries ===== @@ -360,11 +353,11 @@ Input [1]: [d_date_sk#17] (61) BroadcastExchange Input [1]: [d_date_sk#17] -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=4] Subquery:2 Hosting operator id = 4 Hosting Expression = ws_sold_date_sk#10 IN dynamicpruning#4 -Subquery:3 Hosting operator id = 29 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#25 +Subquery:3 Hosting operator id = 28 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#25 BroadcastExchange (66) +- * ColumnarToRow (65) +- CometProject (64) @@ -373,18 +366,18 @@ BroadcastExchange (66) (62) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#31, d_month_seq#41] +Output [2]: [d_date_sk#31, d_month_seq#32] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct (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)) +Input [2]: [d_date_sk#31, d_month_seq#32] +Condition : (((isnotnull(d_month_seq#32) AND (d_month_seq#32 >= Subquery scalar-subquery#33, [id=#34])) AND (d_month_seq#32 <= Subquery scalar-subquery#35, [id=#36])) AND isnotnull(d_date_sk#31)) (64) CometProject -Input [2]: [d_date_sk#31, d_month_seq#41] +Input [2]: [d_date_sk#31, d_month_seq#32] Arguments: [d_date_sk#31], [d_date_sk#31] (65) ColumnarToRow [codegen id : 1] @@ -392,12 +385,12 @@ Input [1]: [d_date_sk#31] (66) BroadcastExchange Input [1]: [d_date_sk#31] -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:4 Hosting operator id = 63 Hosting Expression = Subquery scalar-subquery#42, [id=#43] -* HashAggregate (73) -+- Exchange (72) - +- * ColumnarToRow (71) +Subquery:4 Hosting operator id = 63 Hosting Expression = Subquery scalar-subquery#33, [id=#34] +* ColumnarToRow (73) ++- CometHashAggregate (72) + +- CometColumnarExchange (71) +- CometHashAggregate (70) +- CometProject (69) +- CometFilter (68) @@ -405,43 +398,41 @@ Subquery:4 Hosting operator id = 63 Hosting Expression = Subquery scalar-subquer (67) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#46, d_year#47, d_moy#48] +Output [3]: [d_month_seq#42, d_year#43, d_moy#44] 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 (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)) +Input [3]: [d_month_seq#42, d_year#43, d_moy#44] +Condition : (((isnotnull(d_year#43) AND isnotnull(d_moy#44)) AND (d_year#43 = 1998)) AND (d_moy#44 = 12)) (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] +Input [3]: [d_month_seq#42, d_year#43, d_moy#44] +Arguments: [(d_month_seq + 1)#45], [(d_month_seq#42 + 1) AS (d_month_seq + 1)#45] (70) CometHashAggregate -Input [1]: [(d_month_seq + 1)#49] -Keys [1]: [(d_month_seq + 1)#49] +Input [1]: [(d_month_seq + 1)#45] +Keys [1]: [(d_month_seq + 1)#45] Functions: [] -(71) ColumnarToRow [codegen id : 1] -Input [1]: [(d_month_seq + 1)#49] +(71) CometColumnarExchange +Input [1]: [(d_month_seq + 1)#45] +Arguments: hashpartitioning((d_month_seq + 1)#45, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(72) Exchange -Input [1]: [(d_month_seq + 1)#49] -Arguments: hashpartitioning((d_month_seq + 1)#49, 5), ENSURE_REQUIREMENTS, [plan_id=9] - -(73) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 1)#49] -Keys [1]: [(d_month_seq + 1)#49] +(72) CometHashAggregate +Input [1]: [(d_month_seq + 1)#45] +Keys [1]: [(d_month_seq + 1)#45] Functions: [] -Aggregate Attributes: [] -Results [1]: [(d_month_seq + 1)#49] -Subquery:5 Hosting operator id = 63 Hosting Expression = Subquery scalar-subquery#44, [id=#45] -* HashAggregate (80) -+- Exchange (79) - +- * ColumnarToRow (78) +(73) ColumnarToRow [codegen id : 1] +Input [1]: [(d_month_seq + 1)#45] + +Subquery:5 Hosting operator id = 63 Hosting Expression = Subquery scalar-subquery#35, [id=#36] +* ColumnarToRow (80) ++- CometHashAggregate (79) + +- CometColumnarExchange (78) +- CometHashAggregate (77) +- CometProject (76) +- CometFilter (75) @@ -449,37 +440,39 @@ Subquery:5 Hosting operator id = 63 Hosting Expression = Subquery scalar-subquer (74) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#50, d_year#51, d_moy#52] +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 (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)) +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)) (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] +Input [3]: [d_month_seq#46, d_year#47, d_moy#48] +Arguments: [(d_month_seq + 3)#49], [(d_month_seq#46 + 3) AS (d_month_seq + 3)#49] (77) CometHashAggregate -Input [1]: [(d_month_seq + 3)#53] -Keys [1]: [(d_month_seq + 3)#53] +Input [1]: [(d_month_seq + 3)#49] +Keys [1]: [(d_month_seq + 3)#49] Functions: [] -(78) ColumnarToRow [codegen id : 1] -Input [1]: [(d_month_seq + 3)#53] +(78) CometColumnarExchange +Input [1]: [(d_month_seq + 3)#49] +Arguments: hashpartitioning((d_month_seq + 3)#49, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(79) Exchange -Input [1]: [(d_month_seq + 3)#53] -Arguments: hashpartitioning((d_month_seq + 3)#53, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(80) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 3)#53] -Keys [1]: [(d_month_seq + 3)#53] +(79) CometHashAggregate +Input [1]: [(d_month_seq + 3)#49] +Keys [1]: [(d_month_seq + 3)#49] Functions: [] -Aggregate Attributes: [] -Results [1]: [(d_month_seq + 3)#53] + +(80) ColumnarToRow [codegen id : 1] +Input [1]: [(d_month_seq + 3)#49] + +Subquery:6 Hosting operator id = 44 Hosting Expression = ReusedSubquery Subquery scalar-subquery#33, [id=#34] + +Subquery:7 Hosting operator id = 44 Hosting Expression = ReusedSubquery Subquery scalar-subquery#35, [id=#36] 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 2193d875c..d3a89a6c6 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,115 +1,97 @@ -TakeOrderedAndProject [segment,num_customers,segment_base] - WholeStageCodegen (8) - HashAggregate [segment,count] [count(1),num_customers,segment_base,count] - InputAdapter - Exchange [segment] #1 - 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 (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] - Project [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [ca_county,ca_state,s_county,s_state] - Project [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - HashAggregate [c_customer_sk,c_current_addr_sk] - InputAdapter - Exchange [c_customer_sk,c_current_addr_sk] #3 +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [segment,num_customers,segment_base] + CometHashAggregate [segment,num_customers,segment_base,count,count(1)] + CometColumnarExchange [segment] #1 + CometHashAggregate [segment,count] + CometHashAggregate [segment,c_customer_sk,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange [c_customer_sk] #2 + CometHashAggregate [c_customer_sk,sum,ss_ext_sales_price] + CometProject [c_customer_sk,ss_ext_sales_price] + CometBroadcastHashJoin [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometProject [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state,s_county,s_state] + CometProject [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state] + CometBroadcastHashJoin [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk,ca_address_sk,ca_county,ca_state] + CometProject [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_addr_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + CometHashAggregate [c_customer_sk,c_current_addr_sk] + CometColumnarExchange [c_customer_sk,c_current_addr_sk] #3 + CometHashAggregate [c_customer_sk,c_current_addr_sk] + CometProject [c_customer_sk,c_current_addr_sk] + CometBroadcastHashJoin [customer_sk,c_customer_sk,c_current_addr_sk] + CometProject [customer_sk] + CometBroadcastHashJoin [sold_date_sk,customer_sk,d_date_sk] + CometProject [sold_date_sk,customer_sk] + CometBroadcastHashJoin [sold_date_sk,customer_sk,item_sk,i_item_sk] + CometUnion [sold_date_sk,customer_sk,item_sk] + CometProject [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] [sold_date_sk,customer_sk,item_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [i_item_sk] #5 + CometProject [i_item_sk] + CometFilter [i_item_sk,i_class,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #7 + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] #8 + CometFilter [ss_customer_sk,ss_ext_sales_price,ss_sold_date_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 #9 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,c_current_addr_sk] - CometProject [customer_sk] - CometBroadcastHashJoin [sold_date_sk,customer_sk,d_date_sk] - CometProject [sold_date_sk,customer_sk] - CometBroadcastHashJoin [sold_date_sk,customer_sk,item_sk,i_item_sk] - CometUnion [sold_date_sk,customer_sk,item_sk] - CometProject [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [i_item_sk] #5 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_class,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #7 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk,ss_ext_sales_price,ss_sold_date_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 #9 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + Subquery #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #3 - WholeStageCodegen (2) - HashAggregate [(d_month_seq + 1)] - InputAdapter - Exchange [(d_month_seq + 1)] #10 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 1)] - CometProject [d_month_seq] [(d_month_seq + 1)] - CometFilter [d_month_seq,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - Subquery #4 - WholeStageCodegen (2) - HashAggregate [(d_month_seq + 3)] - InputAdapter - Exchange [(d_month_seq + 3)] #11 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 3)] - CometProject [d_month_seq] [(d_month_seq + 3)] - CometFilter [d_month_seq,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 - 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 #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] #9 + CometHashAggregate [(d_month_seq + 1)] + CometColumnarExchange [(d_month_seq + 1)] #10 + CometHashAggregate [(d_month_seq + 1)] + CometProject [d_month_seq] [(d_month_seq + 1)] + CometFilter [d_month_seq,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + Subquery #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [(d_month_seq + 3)] + CometColumnarExchange [(d_month_seq + 3)] #11 + CometHashAggregate [(d_month_seq + 3)] + CometProject [d_month_seq] [(d_month_seq + 3)] + CometFilter [d_month_seq,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 [ca_address_sk,ca_county,ca_state] #12 + CometFilter [ca_address_sk,ca_county,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state] + CometBroadcastExchange [s_county,s_state] #13 + CometFilter [s_county,s_state] + CometScan parquet spark_catalog.default.store [s_county,s_state] + CometBroadcastExchange [d_date_sk] #14 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + ReusedSubquery [(d_month_seq + 1)] #3 + ReusedSubquery [(d_month_seq + 3)] #4 + 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/q55/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/explain.txt index 00b14b96b..ee250ed9b 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,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (19) -+- * HashAggregate (18) - +- Exchange (17) - +- * ColumnarToRow (16) +* ColumnarToRow (19) ++- CometTakeOrderedAndProject (18) + +- CometHashAggregate (17) + +- CometColumnarExchange (16) +- CometHashAggregate (15) +- CometProject (14) +- CometBroadcastHashJoin (13) @@ -93,21 +93,19 @@ 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))] -(16) ColumnarToRow [codegen id : 1] +(16) CometColumnarExchange Input [3]: [i_brand#9, i_brand_id#8, sum#11] +Arguments: hashpartitioning(i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(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] - -(18) HashAggregate [codegen id : 2] +(17) CometHashAggregate 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))#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] -(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] +(18) CometTakeOrderedAndProject +Input [3]: [brand_id#12, brand#13, ext_price#14] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#14 DESC NULLS LAST,brand_id#12 ASC NULLS FIRST], output=[brand_id#12,brand#13,ext_price#14]), [brand_id#12, brand#13, ext_price#14], 100, [ext_price#14 DESC NULLS LAST, brand_id#12 ASC NULLS FIRST], [brand_id#12, brand#13, ext_price#14] + +(19) ColumnarToRow [codegen id : 1] +Input [3]: [brand_id#12, brand#13, ext_price#14] 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 fc38884c1..45200509f 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,23 +1,21 @@ -TakeOrderedAndProject [ext_price,brand_id,brand] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_brand,i_brand_id,sum,ss_ext_sales_price] - CometProject [ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] - CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [brand_id,brand,ext_price] + CometHashAggregate [brand_id,brand,ext_price,i_brand,i_brand_id,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange [i_brand,i_brand_id] #1 + CometHashAggregate [i_brand,i_brand_id,sum,ss_ext_sales_price] + CometProject [ss_ext_sales_price,i_brand_id,i_brand] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 + CometProject [i_item_sk,i_brand_id,i_brand] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] + 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 74977244e..8061fb91e 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 @@ -1,67 +1,65 @@ == Physical Plan == -TakeOrderedAndProject (63) -+- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- Union (59) - :- * HashAggregate (28) - : +- Exchange (27) - : +- * 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) - : +- * 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) - +- * 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) +* ColumnarToRow (61) ++- CometTakeOrderedAndProject (60) + +- CometHashAggregate (59) + +- CometColumnarExchange (58) + +- CometHashAggregate (57) + +- CometUnion (56) + :- CometHashAggregate (27) + : +- CometColumnarExchange (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) + :- CometHashAggregate (41) + : +- CometColumnarExchange (40) + : +- CometHashAggregate (39) + : +- CometProject (38) + : +- CometBroadcastHashJoin (37) + : :- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometProject (32) + : : : +- CometBroadcastHashJoin (31) + : : : :- CometFilter (29) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (28) + : : : +- ReusedExchange (30) + : : +- ReusedExchange (33) + : +- ReusedExchange (36) + +- CometHashAggregate (55) + +- CometColumnarExchange (54) + +- CometHashAggregate (53) + +- CometProject (52) + +- CometBroadcastHashJoin (51) + :- CometProject (49) + : +- CometBroadcastHashJoin (48) + : :- CometProject (46) + : : +- CometBroadcastHashJoin (45) + : : :- CometFilter (43) + : : : +- CometScan parquet spark_catalog.default.web_sales (42) + : : +- ReusedExchange (44) + : +- ReusedExchange (47) + +- ReusedExchange (50) (1) Scan parquet spark_catalog.default.store_sales @@ -185,212 +183,199 @@ 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))] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarExchange Input [2]: [i_item_id#12, sum#15] +Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(27) Exchange -Input [2]: [i_item_id#12, sum#15] -Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=1] - -(28) HashAggregate [codegen id : 2] +(27) CometHashAggregate 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))#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#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] +(28) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_bill_addr_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#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#22)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#20)] PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(30) CometFilter -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)) +(29) CometFilter +Input [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] +Condition : (isnotnull(cs_bill_addr_sk#16) AND isnotnull(cs_item_sk#17)) -(31) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#23] +(30) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#21] -(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 +(31) CometBroadcastHashJoin +Left output [4]: [cs_bill_addr_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 -(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] +(32) CometProject +Input [5]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19, d_date_sk#21] +Arguments: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18], [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18] -(34) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#24] +(33) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#22] -(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 +(34) CometBroadcastHashJoin +Left output [3]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18] +Right output [1]: [ca_address_sk#22] +Arguments: [cs_bill_addr_sk#16], [ca_address_sk#22], 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] +(35) CometProject +Input [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, ca_address_sk#22] +Arguments: [cs_item_sk#17, cs_ext_sales_price#18], [cs_item_sk#17, cs_ext_sales_price#18] -(37) ReusedExchange [Reuses operator id: 22] -Output [2]: [i_item_sk#25, i_item_id#26] +(36) ReusedExchange [Reuses operator id: 22] +Output [2]: [i_item_sk#23, i_item_id#24] -(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 +(37) CometBroadcastHashJoin +Left output [2]: [cs_item_sk#17, cs_ext_sales_price#18] +Right output [2]: [i_item_sk#23, i_item_id#24] +Arguments: [cs_item_sk#17], [i_item_sk#23], 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] +(38) CometProject +Input [4]: [cs_item_sk#17, cs_ext_sales_price#18, i_item_sk#23, i_item_id#24] +Arguments: [cs_ext_sales_price#18, i_item_id#24], [cs_ext_sales_price#18, i_item_id#24] -(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))] +(39) CometHashAggregate +Input [2]: [cs_ext_sales_price#18, i_item_id#24] +Keys [1]: [i_item_id#24] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#18))] -(41) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_id#26, sum#27] +(40) CometColumnarExchange +Input [2]: [i_item_id#24, sum#25] +Arguments: hashpartitioning(i_item_id#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(42) Exchange -Input [2]: [i_item_id#26, sum#27] -Arguments: hashpartitioning(i_item_id#26, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(41) CometHashAggregate +Input [2]: [i_item_id#24, sum#25] +Keys [1]: [i_item_id#24] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#18))] -(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] +(42) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] 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#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#30)] 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)) +(43) CometFilter +Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] +Condition : (isnotnull(ws_bill_addr_sk#27) AND isnotnull(ws_item_sk#26)) -(46) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#35] +(44) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#31] -(47) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] -Right output [1]: [d_date_sk#35] -Arguments: [ws_sold_date_sk#33], [d_date_sk#35], Inner, BuildRight +(45) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] +Right output [1]: [d_date_sk#31] +Arguments: [ws_sold_date_sk#29], [d_date_sk#31], Inner, BuildRight -(48) CometProject -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] -Arguments: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32], [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] +(46) CometProject +Input [5]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29, d_date_sk#31] +Arguments: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28], [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28] -(49) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#36] +(47) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#32] -(50) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] -Right output [1]: [ca_address_sk#36] -Arguments: [ws_bill_addr_sk#31], [ca_address_sk#36], Inner, BuildRight +(48) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28] +Right output [1]: [ca_address_sk#32] +Arguments: [ws_bill_addr_sk#27], [ca_address_sk#32], Inner, BuildRight -(51) CometProject -Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ca_address_sk#36] -Arguments: [ws_item_sk#30, ws_ext_sales_price#32], [ws_item_sk#30, ws_ext_sales_price#32] +(49) CometProject +Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ca_address_sk#32] +Arguments: [ws_item_sk#26, ws_ext_sales_price#28], [ws_item_sk#26, ws_ext_sales_price#28] -(52) ReusedExchange [Reuses operator id: 22] -Output [2]: [i_item_sk#37, i_item_id#38] +(50) ReusedExchange [Reuses operator id: 22] +Output [2]: [i_item_sk#33, i_item_id#34] -(53) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_ext_sales_price#32] -Right output [2]: [i_item_sk#37, i_item_id#38] -Arguments: [ws_item_sk#30], [i_item_sk#37], Inner, BuildRight +(51) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#26, ws_ext_sales_price#28] +Right output [2]: [i_item_sk#33, i_item_id#34] +Arguments: [ws_item_sk#26], [i_item_sk#33], Inner, BuildRight -(54) CometProject -Input [4]: [ws_item_sk#30, ws_ext_sales_price#32, i_item_sk#37, i_item_id#38] -Arguments: [ws_ext_sales_price#32, i_item_id#38], [ws_ext_sales_price#32, i_item_id#38] +(52) CometProject +Input [4]: [ws_item_sk#26, ws_ext_sales_price#28, i_item_sk#33, i_item_id#34] +Arguments: [ws_ext_sales_price#28, i_item_id#34], [ws_ext_sales_price#28, i_item_id#34] -(55) CometHashAggregate -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))] +(53) CometHashAggregate +Input [2]: [ws_ext_sales_price#28, i_item_id#34] +Keys [1]: [i_item_id#34] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#28))] -(56) ColumnarToRow [codegen id : 5] -Input [2]: [i_item_id#38, sum#39] +(54) CometColumnarExchange +Input [2]: [i_item_id#34, sum#35] +Arguments: hashpartitioning(i_item_id#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(57) Exchange -Input [2]: [i_item_id#38, sum#39] -Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(58) HashAggregate [codegen id : 6] -Input [2]: [i_item_id#38, sum#39] -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))#40] -Results [2]: [i_item_id#38, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#32))#40,17,2) AS total_sales#41] +(55) CometHashAggregate +Input [2]: [i_item_id#34, sum#35] +Keys [1]: [i_item_id#34] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#28))] -(59) Union +(56) CometUnion +Child 0 Input [2]: [i_item_id#12, total_sales#36] +Child 1 Input [2]: [i_item_id#24, total_sales#37] +Child 2 Input [2]: [i_item_id#34, total_sales#38] -(60) HashAggregate [codegen id : 7] -Input [2]: [i_item_id#12, total_sales#17] +(57) CometHashAggregate +Input [2]: [i_item_id#12, total_sales#36] Keys [1]: [i_item_id#12] -Functions [1]: [partial_sum(total_sales#17)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [3]: [i_item_id#12, sum#44, isEmpty#45] +Functions [1]: [partial_sum(total_sales#36)] -(61) Exchange -Input [3]: [i_item_id#12, sum#44, isEmpty#45] -Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(58) CometColumnarExchange +Input [3]: [i_item_id#12, sum#39, isEmpty#40] +Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(62) HashAggregate [codegen id : 8] -Input [3]: [i_item_id#12, sum#44, isEmpty#45] +(59) CometHashAggregate +Input [3]: [i_item_id#12, sum#39, isEmpty#40] Keys [1]: [i_item_id#12] -Functions [1]: [sum(total_sales#17)] -Aggregate Attributes [1]: [sum(total_sales#17)#46] -Results [2]: [i_item_id#12, sum(total_sales#17)#46 AS total_sales#47] +Functions [1]: [sum(total_sales#36)] + +(60) CometTakeOrderedAndProject +Input [2]: [i_item_id#12, total_sales#41] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_sales#41 ASC NULLS FIRST], output=[i_item_id#12,total_sales#41]), [i_item_id#12, total_sales#41], 100, [total_sales#41 ASC NULLS FIRST], [i_item_id#12, total_sales#41] -(63) TakeOrderedAndProject -Input [2]: [i_item_id#12, total_sales#47] -Arguments: 100, [total_sales#47 ASC NULLS FIRST], [i_item_id#12, total_sales#47] +(61) ColumnarToRow [codegen id : 1] +Input [2]: [i_item_id#12, total_sales#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (68) -+- * ColumnarToRow (67) - +- CometProject (66) - +- CometFilter (65) - +- CometScan parquet spark_catalog.default.date_dim (64) +BroadcastExchange (66) ++- * ColumnarToRow (65) + +- CometProject (64) + +- CometFilter (63) + +- CometScan parquet spark_catalog.default.date_dim (62) -(64) Scan parquet spark_catalog.default.date_dim +(62) Scan parquet spark_catalog.default.date_dim 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 +(63) 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)) -(66) CometProject +(64) CometProject 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] +(65) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(68) BroadcastExchange +(66) BroadcastExchange Input [1]: [d_date_sk#6] 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#21 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 28 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#29 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 0b48046fd..4bb76eecc 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,89 +1,73 @@ -TakeOrderedAndProject [total_sales,i_item_id] - WholeStageCodegen (8) - HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - InputAdapter - Exchange [i_item_id] #1 - WholeStageCodegen (7) - HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (2) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_item_id] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,sum,ss_ext_sales_price] - CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,total_sales] + CometHashAggregate [i_item_id,total_sales,sum,isEmpty,sum(total_sales)] + CometColumnarExchange [i_item_id] #1 + CometHashAggregate [i_item_id,sum,isEmpty,total_sales] + CometUnion [i_item_id,total_sales] + CometHashAggregate [i_item_id,total_sales,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange [i_item_id] #2 + CometHashAggregate [i_item_id,sum,ss_ext_sales_price] + CometProject [ss_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ca_address_sk] #5 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [i_item_id] #7 - CometProject [i_item_id] - CometFilter [i_item_id,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] #8 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,sum,cs_ext_sales_price] - CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] - CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] - CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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 - 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] #9 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,sum,ws_ext_sales_price] - CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] - CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] - CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ca_address_sk] #5 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange [i_item_id] #7 + CometProject [i_item_id] + CometFilter [i_item_id,i_color] + CometScan parquet spark_catalog.default.item [i_item_id,i_color] + CometHashAggregate [i_item_id,total_sales,sum,sum(UnscaledValue(cs_ext_sales_price))] + CometColumnarExchange [i_item_id] #8 + CometHashAggregate [i_item_id,sum,cs_ext_sales_price] + CometProject [cs_ext_sales_price,i_item_id] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] + CometProject [cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] + CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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 + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #6 + CometHashAggregate [i_item_id,total_sales,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometColumnarExchange [i_item_id] #9 + CometHashAggregate [i_item_id,sum,ws_ext_sales_price] + CometProject [ws_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] + CometProject [ws_item_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] + CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 + 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 197603ca3..b5372cffc 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 @@ -2,18 +2,18 @@ TakeOrderedAndProject (45) +- * Project (44) +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) + :- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) : :- * Project (28) : : +- * Filter (27) : : +- Window (26) : : +- * Filter (25) : : +- Window (24) - : : +- * Sort (23) - : : +- Exchange (22) - : : +- * HashAggregate (21) - : : +- Exchange (20) - : : +- * ColumnarToRow (19) + : : +- * ColumnarToRow (23) + : : +- CometSort (22) + : : +- CometColumnarExchange (21) + : : +- CometHashAggregate (20) + : : +- CometColumnarExchange (19) : : +- CometHashAggregate (18) : : +- CometProject (17) : : +- CometBroadcastHashJoin (16) @@ -32,18 +32,18 @@ TakeOrderedAndProject (45) : : +- CometBroadcastExchange (15) : : +- CometFilter (14) : : +- CometScan parquet spark_catalog.default.call_center (13) - : +- BroadcastExchange (35) - : +- * Project (34) - : +- Window (33) - : +- * Sort (32) - : +- Exchange (31) - : +- * HashAggregate (30) - : +- ReusedExchange (29) + : +- BroadcastExchange (34) + : +- * Project (33) + : +- Window (32) + : +- * ColumnarToRow (31) + : +- CometSort (30) + : +- ReusedExchange (29) +- BroadcastExchange (42) +- * Project (41) +- Window (40) - +- * Sort (39) - +- ReusedExchange (38) + +- * ColumnarToRow (39) + +- CometSort (38) + +- ReusedExchange (37) (1) Scan parquet spark_catalog.default.item @@ -135,120 +135,113 @@ Input [6]: [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_n 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))] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(20) Exchange -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 : 2] +(20) CometHashAggregate 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))#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#16, _w0#17] -Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(21) CometColumnarExchange +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16] +Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(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 +(22) CometSort +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16] +Arguments: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16], [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] + +(23) ColumnarToRow [codegen id : 1] +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16] (24) Window -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] +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16] +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#17], [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 : 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] +(25) Filter [codegen id : 2] +Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16, rn#17] 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#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 : 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 : 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#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum#25] - -(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#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 : 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#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 : 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#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 : 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)] +Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16, rn#17] +Arguments: [avg(_w0#16) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#18], [i_category#3, i_brand#2, cc_name#13, d_year#10] + +(27) Filter [codegen id : 7] +Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16, rn#17, avg_monthly_sales#18] +Condition : ((isnotnull(avg_monthly_sales#18) AND (avg_monthly_sales#18 > 0.000000)) AND CASE WHEN (avg_monthly_sales#18 > 0.000000) THEN ((abs((sum_sales#15 - avg_monthly_sales#18)) / avg_monthly_sales#18) > 0.1000000000000000) END) + +(28) Project [codegen id : 7] +Output [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, avg_monthly_sales#18, rn#17] +Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16, rn#17, avg_monthly_sales#18] + +(29) ReusedExchange [Reuses operator id: 21] +Output [6]: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15] + +(30) CometSort +Input [6]: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15] +Arguments: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15], [i_category#19 ASC NULLS FIRST, i_brand#20 ASC NULLS FIRST, cc_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] + +(31) ColumnarToRow [codegen id : 3] +Input [6]: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15] + +(32) Window +Input [6]: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15] +Arguments: [rank(d_year#22, d_moy#23) windowspecdefinition(i_category#19, i_brand#20, cc_name#21, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#24], [i_category#19, i_brand#20, cc_name#21], [d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] + +(33) Project [codegen id : 4] +Output [5]: [i_category#19, i_brand#20, cc_name#21, sum_sales#15 AS sum_sales#25, rn#24] +Input [7]: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15, rn#24] + +(34) BroadcastExchange +Input [5]: [i_category#19, i_brand#20, cc_name#21, sum_sales#25, rn#24] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=3] + +(35) BroadcastHashJoin [codegen id : 7] +Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#17] +Right keys [4]: [i_category#19, i_brand#20, cc_name#21, (rn#24 + 1)] Join type: Inner Join condition: None -(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] +(36) Project [codegen id : 7] +Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, avg_monthly_sales#18, rn#17, sum_sales#25] +Input [13]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, avg_monthly_sales#18, rn#17, i_category#19, i_brand#20, cc_name#21, sum_sales#25, rn#24] + +(37) ReusedExchange [Reuses operator id: 21] +Output [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#15] -(38) ReusedExchange [Reuses operator id: 31] -Output [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#16] +(38) CometSort +Input [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#15] +Arguments: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#15], [i_category#26 ASC NULLS FIRST, i_brand#27 ASC NULLS FIRST, cc_name#28 ASC NULLS FIRST, d_year#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST] -(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 +(39) ColumnarToRow [codegen id : 5] +Input [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#15] (40) Window -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] +Input [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#15] +Arguments: [rank(d_year#29, d_moy#30) windowspecdefinition(i_category#26, i_brand#27, cc_name#28, d_year#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#31], [i_category#26, i_brand#27, cc_name#28], [d_year#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST] -(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] +(41) Project [codegen id : 6] +Output [5]: [i_category#26, i_brand#27, cc_name#28, sum_sales#15 AS sum_sales#32, rn#31] +Input [7]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#15, rn#31] (42) BroadcastExchange -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] +Input [5]: [i_category#26, i_brand#27, cc_name#28, sum_sales#32, rn#31] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=4] -(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)] +(43) BroadcastHashJoin [codegen id : 7] +Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#17] +Right keys [4]: [i_category#26, i_brand#27, cc_name#28, (rn#31 - 1)] Join type: Inner Join condition: None -(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] +(44) Project [codegen id : 7] +Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#18, sum_sales#15, sum_sales#25 AS psum#33, sum_sales#32 AS nsum#34] +Input [14]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, avg_monthly_sales#18, rn#17, sum_sales#25, i_category#26, i_brand#27, cc_name#28, sum_sales#32, rn#31] (45) TakeOrderedAndProject -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] +Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#18, sum_sales#15, psum#33, nsum#34] +Arguments: 100, [(sum_sales#15 - avg_monthly_sales#18) 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#18, sum_sales#15, psum#33, nsum#34] ===== Subqueries ===== @@ -275,6 +268,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=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] 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 fb2b09b9d..454d12fae 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 (13) + WholeStageCodegen (7) 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,67 +8,61 @@ 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 (4) + WholeStageCodegen (2) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (3) - Sort [i_category,i_brand,cc_name,d_year,d_moy] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [i_category,i_brand,cc_name] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum,cs_sales_price] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometFilter [i_item_sk,i_brand,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 - CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,cc_name] #1 + CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum,sum(UnscaledValue(cs_sales_price))] + CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum,cs_sales_price] + CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_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_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 + CometFilter [cc_call_center_sk,cc_name] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter BroadcastExchange #7 - WholeStageCodegen (8) + WholeStageCodegen (4) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (7) - Sort [i_category,i_brand,cc_name,d_year,d_moy] + WholeStageCodegen (3) + ColumnarToRow InputAdapter - 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 + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #1 InputAdapter - BroadcastExchange #9 - WholeStageCodegen (12) + BroadcastExchange #8 + WholeStageCodegen (6) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (11) - Sort [i_category,i_brand,cc_name,d_year,d_moy] + WholeStageCodegen (5) + ColumnarToRow InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #1 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 9d5fe57a2..06e3fb959 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,57 +1,55 @@ == Physical Plan == -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) +* ColumnarToRow (51) ++- CometTakeOrderedAndProject (50) + +- CometProject (49) + +- CometBroadcastHashJoin (48) + :- CometProject (34) + : +- CometBroadcastHashJoin (33) + : :- CometFilter (19) + : : +- CometHashAggregate (18) + : : +- CometColumnarExchange (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) + : +- CometBroadcastExchange (32) + : +- CometFilter (31) + : +- CometHashAggregate (30) + : +- CometColumnarExchange (29) + : +- CometHashAggregate (28) + : +- CometProject (27) + : +- CometBroadcastHashJoin (26) + : :- CometProject (24) + : : +- CometBroadcastHashJoin (23) + : : :- CometFilter (21) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (20) + : : +- ReusedExchange (22) + : +- ReusedExchange (25) + +- CometBroadcastExchange (47) + +- CometFilter (46) + +- CometHashAggregate (45) + +- CometColumnarExchange (44) + +- CometHashAggregate (43) + +- CometProject (42) + +- CometBroadcastHashJoin (41) + :- CometProject (39) + : +- CometBroadcastHashJoin (38) + : :- CometFilter (36) + : : +- CometScan parquet spark_catalog.default.web_sales (35) + : +- ReusedExchange (37) + +- ReusedExchange (40) (1) Scan parquet spark_catalog.default.store_sales @@ -101,7 +99,7 @@ ReadSchema: struct Input [2]: [d_date_sk#7, d_date#8] Condition : isnotnull(d_date_sk#7) -(10) ReusedExchange [Reuses operator id: 59] +(10) ReusedExchange [Reuses operator id: 57] Output [1]: [d_date#9] (11) CometBroadcastHashJoin @@ -131,262 +129,248 @@ 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))] -(17) ColumnarToRow [codegen id : 1] +(17) CometColumnarExchange Input [2]: [i_item_id#6, sum#10] +Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(18) Exchange -Input [2]: [i_item_id#6, sum#10] -Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [plan_id=1] - -(19) HashAggregate [codegen id : 6] +(18) CometHashAggregate 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))#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] -(20) Filter [codegen id : 6] -Input [2]: [item_id#12, ss_item_rev#13] -Condition : isnotnull(ss_item_rev#13) +(19) CometFilter +Input [2]: [item_id#11, ss_item_rev#12] +Condition : isnotnull(ss_item_rev#12) -(21) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] +(20) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_item_sk#13, cs_ext_sales_price#14, cs_sold_date_sk#15] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] +PartitionFilters: [isnotnull(cs_sold_date_sk#15), dynamicpruningexpression(cs_sold_date_sk#15 IN dynamicpruning#16)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(22) CometFilter -Input [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] -Condition : isnotnull(cs_item_sk#14) - -(23) ReusedExchange [Reuses operator id: 5] -Output [2]: [i_item_sk#18, i_item_id#19] - -(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 - -(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] - -(26) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#20] - -(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 - -(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) - -(34) BroadcastExchange -Input [2]: [item_id#23, cs_item_rev#24] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] - -(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#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))) - -(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] - -(37) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] +(21) CometFilter +Input [3]: [cs_item_sk#13, cs_ext_sales_price#14, cs_sold_date_sk#15] +Condition : isnotnull(cs_item_sk#13) + +(22) ReusedExchange [Reuses operator id: 5] +Output [2]: [i_item_sk#17, i_item_id#18] + +(23) CometBroadcastHashJoin +Left output [3]: [cs_item_sk#13, cs_ext_sales_price#14, cs_sold_date_sk#15] +Right output [2]: [i_item_sk#17, i_item_id#18] +Arguments: [cs_item_sk#13], [i_item_sk#17], Inner, BuildRight + +(24) CometProject +Input [5]: [cs_item_sk#13, cs_ext_sales_price#14, cs_sold_date_sk#15, i_item_sk#17, i_item_id#18] +Arguments: [cs_ext_sales_price#14, cs_sold_date_sk#15, i_item_id#18], [cs_ext_sales_price#14, cs_sold_date_sk#15, i_item_id#18] + +(25) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_date_sk#19] + +(26) CometBroadcastHashJoin +Left output [3]: [cs_ext_sales_price#14, cs_sold_date_sk#15, i_item_id#18] +Right output [1]: [d_date_sk#19] +Arguments: [cs_sold_date_sk#15], [d_date_sk#19], Inner, BuildRight + +(27) CometProject +Input [4]: [cs_ext_sales_price#14, cs_sold_date_sk#15, i_item_id#18, d_date_sk#19] +Arguments: [cs_ext_sales_price#14, i_item_id#18], [cs_ext_sales_price#14, i_item_id#18] + +(28) CometHashAggregate +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))] + +(29) CometColumnarExchange +Input [2]: [i_item_id#18, sum#20] +Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(30) CometHashAggregate +Input [2]: [i_item_id#18, sum#20] +Keys [1]: [i_item_id#18] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#14))] + +(31) CometFilter +Input [2]: [item_id#21, cs_item_rev#22] +Condition : isnotnull(cs_item_rev#22) + +(32) CometBroadcastExchange +Input [2]: [item_id#21, cs_item_rev#22] +Arguments: [item_id#21, cs_item_rev#22] + +(33) CometBroadcastHashJoin +Left output [2]: [item_id#11, ss_item_rev#12] +Right output [2]: [item_id#21, cs_item_rev#22] +Arguments: [item_id#11], [item_id#21], Inner, ((((cast(ss_item_rev#12 as decimal(19,3)) >= (0.9 * cs_item_rev#22)) AND (cast(ss_item_rev#12 as decimal(20,3)) <= (1.1 * cs_item_rev#22))) AND (cast(cs_item_rev#22 as decimal(19,3)) >= (0.9 * ss_item_rev#12))) AND (cast(cs_item_rev#22 as decimal(20,3)) <= (1.1 * ss_item_rev#12))), BuildRight + +(34) CometProject +Input [4]: [item_id#11, ss_item_rev#12, item_id#21, cs_item_rev#22] +Arguments: [item_id#11, ss_item_rev#12, cs_item_rev#22], [item_id#11, ss_item_rev#12, cs_item_rev#22] + +(35) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_item_sk#23, ws_ext_sales_price#24, ws_sold_date_sk#25] 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#25), dynamicpruningexpression(ws_sold_date_sk#25 IN dynamicpruning#26)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(38) CometFilter -Input [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] -Condition : isnotnull(ws_item_sk#25) +(36) CometFilter +Input [3]: [ws_item_sk#23, ws_ext_sales_price#24, ws_sold_date_sk#25] +Condition : isnotnull(ws_item_sk#23) + +(37) ReusedExchange [Reuses operator id: 5] +Output [2]: [i_item_sk#27, i_item_id#28] -(39) ReusedExchange [Reuses operator id: 5] -Output [2]: [i_item_sk#29, i_item_id#30] +(38) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#23, ws_ext_sales_price#24, ws_sold_date_sk#25] +Right output [2]: [i_item_sk#27, i_item_id#28] +Arguments: [ws_item_sk#23], [i_item_sk#27], Inner, BuildRight -(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 +(39) CometProject +Input [5]: [ws_item_sk#23, ws_ext_sales_price#24, ws_sold_date_sk#25, i_item_sk#27, i_item_id#28] +Arguments: [ws_ext_sales_price#24, ws_sold_date_sk#25, i_item_id#28], [ws_ext_sales_price#24, ws_sold_date_sk#25, i_item_id#28] -(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] +(40) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_date_sk#29] -(42) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#31] +(41) CometBroadcastHashJoin +Left output [3]: [ws_ext_sales_price#24, ws_sold_date_sk#25, i_item_id#28] +Right output [1]: [d_date_sk#29] +Arguments: [ws_sold_date_sk#25], [d_date_sk#29], Inner, BuildRight -(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 +(42) CometProject +Input [4]: [ws_ext_sales_price#24, ws_sold_date_sk#25, i_item_id#28, d_date_sk#29] +Arguments: [ws_ext_sales_price#24, i_item_id#28], [ws_ext_sales_price#24, i_item_id#28] -(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] +(43) CometHashAggregate +Input [2]: [ws_ext_sales_price#24, i_item_id#28] +Keys [1]: [i_item_id#28] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#24))] + +(44) CometColumnarExchange +Input [2]: [i_item_id#28, sum#30] +Arguments: hashpartitioning(i_item_id#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (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))] - -(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] - -(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))#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] - -(49) Filter [codegen id : 5] -Input [2]: [item_id#34, ws_item_rev#35] -Condition : isnotnull(ws_item_rev#35) - -(50) BroadcastExchange -Input [2]: [item_id#34, ws_item_rev#35] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] - -(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#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))) - -(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] - -(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] +Input [2]: [i_item_id#28, sum#30] +Keys [1]: [i_item_id#28] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#24))] -===== Subqueries ===== +(46) CometFilter +Input [2]: [item_id#31, ws_item_rev#32] +Condition : isnotnull(ws_item_rev#32) -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -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) +(47) CometBroadcastExchange +Input [2]: [item_id#31, ws_item_rev#32] +Arguments: [item_id#31, ws_item_rev#32] +(48) CometBroadcastHashJoin +Left output [3]: [item_id#11, ss_item_rev#12, cs_item_rev#22] +Right output [2]: [item_id#31, ws_item_rev#32] +Arguments: [item_id#11], [item_id#31], Inner, ((((((((cast(ss_item_rev#12 as decimal(19,3)) >= (0.9 * ws_item_rev#32)) AND (cast(ss_item_rev#12 as decimal(20,3)) <= (1.1 * ws_item_rev#32))) AND (cast(cs_item_rev#22 as decimal(19,3)) >= (0.9 * ws_item_rev#32))) AND (cast(cs_item_rev#22 as decimal(20,3)) <= (1.1 * ws_item_rev#32))) AND (cast(ws_item_rev#32 as decimal(19,3)) >= (0.9 * ss_item_rev#12))) AND (cast(ws_item_rev#32 as decimal(20,3)) <= (1.1 * ss_item_rev#12))) AND (cast(ws_item_rev#32 as decimal(19,3)) >= (0.9 * cs_item_rev#22))) AND (cast(ws_item_rev#32 as decimal(20,3)) <= (1.1 * cs_item_rev#22))), BuildRight -(54) Scan parquet spark_catalog.default.date_dim +(49) CometProject +Input [5]: [item_id#11, ss_item_rev#12, cs_item_rev#22, item_id#31, ws_item_rev#32] +Arguments: [item_id#11, ss_item_rev#12, ss_dev#33, cs_item_rev#22, cs_dev#34, ws_item_rev#32, ws_dev#35, average#36], [item_id#11, ss_item_rev#12, (((ss_item_rev#12 / ((ss_item_rev#12 + cs_item_rev#22) + ws_item_rev#32)) / 3) * 100) AS ss_dev#33, cs_item_rev#22, (((cs_item_rev#22 / ((ss_item_rev#12 + cs_item_rev#22) + ws_item_rev#32)) / 3) * 100) AS cs_dev#34, ws_item_rev#32, (((ws_item_rev#32 / ((ss_item_rev#12 + cs_item_rev#22) + ws_item_rev#32)) / 3) * 100) AS ws_dev#35, (((ss_item_rev#12 + cs_item_rev#22) + ws_item_rev#32) / 3) AS average#36] + +(50) CometTakeOrderedAndProject +Input [8]: [item_id#11, ss_item_rev#12, ss_dev#33, cs_item_rev#22, cs_dev#34, ws_item_rev#32, ws_dev#35, average#36] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_id#11 ASC NULLS FIRST,ss_item_rev#12 ASC NULLS FIRST], output=[item_id#11,ss_item_rev#12,ss_dev#33,cs_item_rev#22,cs_dev#34,ws_item_rev#32,ws_dev#35,average#36]), [item_id#11, ss_item_rev#12, ss_dev#33, cs_item_rev#22, cs_dev#34, ws_item_rev#32, ws_dev#35, average#36], 100, [item_id#11 ASC NULLS FIRST, ss_item_rev#12 ASC NULLS FIRST], [item_id#11, ss_item_rev#12, ss_dev#33, cs_item_rev#22, cs_dev#34, ws_item_rev#32, ws_dev#35, average#36] + +(51) ColumnarToRow [codegen id : 1] +Input [8]: [item_id#11, ss_item_rev#12, ss_dev#33, cs_item_rev#22, cs_dev#34, ws_item_rev#32, ws_dev#35, average#36] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 +BroadcastExchange (61) ++- * ColumnarToRow (60) + +- CometProject (59) + +- CometBroadcastHashJoin (58) + :- CometFilter (53) + : +- CometScan parquet spark_catalog.default.date_dim (52) + +- CometBroadcastExchange (57) + +- CometProject (56) + +- CometFilter (55) + +- CometScan parquet spark_catalog.default.date_dim (54) + + +(52) 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 -(55) CometFilter +(53) CometFilter Input [2]: [d_date_sk#7, d_date#8] Condition : isnotnull(d_date_sk#7) -(56) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date#9, d_week_seq#40] +(54) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date#9, d_week_seq#37] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct -(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) CometFilter +Input [2]: [d_date#9, d_week_seq#37] +Condition : (isnotnull(d_week_seq#37) AND (d_week_seq#37 = Subquery scalar-subquery#38, [id=#39])) -(58) CometProject -Input [2]: [d_date#9, d_week_seq#40] +(56) CometProject +Input [2]: [d_date#9, d_week_seq#37] Arguments: [d_date#9], [d_date#9] -(59) CometBroadcastExchange +(57) CometBroadcastExchange Input [1]: [d_date#9] Arguments: [d_date#9] -(60) CometBroadcastHashJoin +(58) 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 -(61) CometProject +(59) CometProject Input [2]: [d_date_sk#7, d_date#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(62) ColumnarToRow [codegen id : 1] +(60) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(63) BroadcastExchange +(61) 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] -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) +Subquery:2 Hosting operator id = 55 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +* ColumnarToRow (65) ++- CometProject (64) + +- CometFilter (63) + +- CometScan parquet spark_catalog.default.date_dim (62) -(64) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date#43, d_week_seq#44] +(62) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date#40, d_week_seq#41] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] ReadSchema: struct -(65) CometFilter -Input [2]: [d_date#43, d_week_seq#44] -Condition : (isnotnull(d_date#43) AND (d_date#43 = 2000-01-03)) +(63) CometFilter +Input [2]: [d_date#40, d_week_seq#41] +Condition : (isnotnull(d_date#40) AND (d_date#40 = 2000-01-03)) -(66) CometProject -Input [2]: [d_date#43, d_week_seq#44] -Arguments: [d_week_seq#44], [d_week_seq#44] +(64) CometProject +Input [2]: [d_date#40, d_week_seq#41] +Arguments: [d_week_seq#41], [d_week_seq#41] -(67) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#44] +(65) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#41] -Subquery:3 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#4 +Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#4 -Subquery:4 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#4 +Subquery:4 Hosting operator id = 35 Hosting Expression = ws_sold_date_sk#25 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 fc7c21e94..326fdcb78 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,89 +1,75 @@ -TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] - 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] - BroadcastHashJoin [item_id,item_id,ss_item_rev,cs_item_rev] - Filter [ss_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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,sum,ss_ext_sales_price] - CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,d_date_sk] - CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id] - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_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 (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] - CometFilter [d_date,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] - 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 [i_item_sk,i_item_id] #4 - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date] #3 - InputAdapter - 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] #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,sum,cs_ext_sales_price] - CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,d_date_sk] - CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id] - CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [i_item_sk,i_item_id] #4 - ReusedExchange [d_date_sk] #5 - InputAdapter - 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] #9 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,sum,ws_ext_sales_price] - CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,d_date_sk] - CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id] - CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [i_item_sk,i_item_id] #4 - ReusedExchange [d_date_sk] #5 +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] + CometProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] + CometBroadcastHashJoin [item_id,ss_item_rev,cs_item_rev,item_id,ws_item_rev] + CometProject [item_id,ss_item_rev,cs_item_rev] + CometBroadcastHashJoin [item_id,ss_item_rev,item_id,cs_item_rev] + CometFilter [item_id,ss_item_rev] + CometHashAggregate [item_id,ss_item_rev,i_item_id,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange [i_item_id] #1 + CometHashAggregate [i_item_id,sum,ss_ext_sales_price] + CometProject [ss_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,d_date_sk] + CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_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 (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date] #3 + CometProject [d_date] + CometFilter [d_date,d_week_seq] + Subquery #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_date,d_week_seq] + 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 [i_item_sk,i_item_id] #4 + CometFilter [i_item_sk,i_item_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + ReusedExchange [d_date] #3 + CometBroadcastExchange [item_id,cs_item_rev] #6 + CometFilter [item_id,cs_item_rev] + CometHashAggregate [item_id,cs_item_rev,i_item_id,sum,sum(UnscaledValue(cs_ext_sales_price))] + CometColumnarExchange [i_item_id] #7 + CometHashAggregate [i_item_id,sum,cs_ext_sales_price] + CometProject [cs_ext_sales_price,i_item_id] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,d_date_sk] + CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [i_item_sk,i_item_id] #4 + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange [item_id,ws_item_rev] #8 + CometFilter [item_id,ws_item_rev] + CometHashAggregate [item_id,ws_item_rev,i_item_id,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometColumnarExchange [i_item_id] #9 + CometHashAggregate [i_item_id,sum,ws_ext_sales_price] + CometProject [ws_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,d_date_sk] + CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + 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 3af7e7296..d3621f8de 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,47 +1,43 @@ == Physical Plan == -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) +* ColumnarToRow (39) ++- CometTakeOrderedAndProject (38) + +- CometProject (37) + +- CometBroadcastHashJoin (36) + :- CometProject (21) + : +- CometBroadcastHashJoin (20) + : :- CometProject (15) + : : +- CometBroadcastHashJoin (14) + : : :- CometHashAggregate (10) + : : : +- CometColumnarExchange (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) + : : +- CometBroadcastExchange (13) + : : +- CometFilter (12) + : : +- CometScan parquet spark_catalog.default.store (11) + : +- CometBroadcastExchange (19) + : +- CometProject (18) + : +- CometFilter (17) + : +- CometScan parquet spark_catalog.default.date_dim (16) + +- CometBroadcastExchange (35) + +- CometProject (34) + +- CometBroadcastHashJoin (33) + :- CometProject (28) + : +- CometBroadcastHashJoin (27) + : :- CometHashAggregate (23) + : : +- ReusedExchange (22) + : +- CometBroadcastExchange (26) + : +- CometFilter (25) + : +- CometScan parquet spark_catalog.default.store (24) + +- CometBroadcastExchange (32) + +- CometProject (31) + +- CometFilter (30) + +- CometScan parquet spark_catalog.default.date_dim (29) (1) Scan parquet spark_catalog.default.store_sales @@ -85,165 +81,144 @@ 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))] -(9) ColumnarToRow [codegen id : 1] +(9) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(11) HashAggregate [codegen id : 8] +(10) CometHashAggregate 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))#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] -(12) Scan parquet spark_catalog.default.store -Output [3]: [s_store_sk#28, s_store_id#29, s_store_name#30] +(11) Scan parquet spark_catalog.default.store +Output [3]: [s_store_sk#14, s_store_id#15, s_store_name#16] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_id)] ReadSchema: struct -(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)) - -(14) ColumnarToRow [codegen id : 2] -Input [3]: [s_store_sk#28, s_store_id#29, s_store_name#30] +(12) CometFilter +Input [3]: [s_store_sk#14, s_store_id#15, s_store_name#16] +Condition : (isnotnull(s_store_sk#14) AND isnotnull(s_store_id#15)) -(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] +(13) CometBroadcastExchange +Input [3]: [s_store_sk#14, s_store_id#15, s_store_name#16] +Arguments: [s_store_sk#14, s_store_id#15, s_store_name#16] -(16) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#28] -Join type: Inner -Join condition: None +(14) CometBroadcastHashJoin +Left output [9]: [d_week_seq#5, ss_store_sk#1, sun_sales#17, mon_sales#18, tue_sales#19, wed_sales#20, thu_sales#21, fri_sales#22, sat_sales#23] +Right output [3]: [s_store_sk#14, s_store_id#15, s_store_name#16] +Arguments: [ss_store_sk#1], [s_store_sk#14], Inner, BuildRight -(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] +(15) CometProject +Input [12]: [d_week_seq#5, ss_store_sk#1, sun_sales#17, mon_sales#18, tue_sales#19, wed_sales#20, thu_sales#21, fri_sales#22, sat_sales#23, s_store_sk#14, s_store_id#15, s_store_name#16] +Arguments: [d_week_seq#5, sun_sales#17, mon_sales#18, tue_sales#19, wed_sales#20, thu_sales#21, fri_sales#22, sat_sales#23, s_store_id#15, s_store_name#16], [d_week_seq#5, sun_sales#17, mon_sales#18, tue_sales#19, wed_sales#20, thu_sales#21, fri_sales#22, sat_sales#23, s_store_id#15, s_store_name#16] -(18) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#31, d_week_seq#32] +(16) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_month_seq#24, d_week_seq#25] 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 -(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)) +(17) CometFilter +Input [2]: [d_month_seq#24, d_week_seq#25] +Condition : (((isnotnull(d_month_seq#24) AND (d_month_seq#24 >= 1212)) AND (d_month_seq#24 <= 1223)) AND isnotnull(d_week_seq#25)) -(20) CometProject -Input [2]: [d_month_seq#31, d_week_seq#32] -Arguments: [d_week_seq#32], [d_week_seq#32] +(18) CometProject +Input [2]: [d_month_seq#24, d_week_seq#25] +Arguments: [d_week_seq#25], [d_week_seq#25] -(21) ColumnarToRow [codegen id : 3] -Input [1]: [d_week_seq#32] +(19) CometBroadcastExchange +Input [1]: [d_week_seq#25] +Arguments: [d_week_seq#25] -(22) BroadcastExchange -Input [1]: [d_week_seq#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(20) CometBroadcastHashJoin +Left output [10]: [d_week_seq#5, sun_sales#17, mon_sales#18, tue_sales#19, wed_sales#20, thu_sales#21, fri_sales#22, sat_sales#23, s_store_id#15, s_store_name#16] +Right output [1]: [d_week_seq#25] +Arguments: [d_week_seq#5], [d_week_seq#25], Inner, BuildRight -(23) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [d_week_seq#5] -Right keys [1]: [d_week_seq#32] -Join type: Inner -Join condition: None +(21) CometProject +Input [11]: [d_week_seq#5, sun_sales#17, mon_sales#18, tue_sales#19, wed_sales#20, thu_sales#21, fri_sales#22, sat_sales#23, s_store_id#15, s_store_name#16, d_week_seq#25] +Arguments: [s_store_name1#26, d_week_seq1#27, s_store_id1#28, sun_sales1#29, mon_sales1#30, tue_sales1#31, wed_sales1#32, thu_sales1#33, fri_sales1#34, sat_sales1#35], [s_store_name#16 AS s_store_name1#26, d_week_seq#5 AS d_week_seq1#27, s_store_id#15 AS s_store_id1#28, sun_sales#17 AS sun_sales1#29, mon_sales#18 AS mon_sales1#30, tue_sales#19 AS tue_sales1#31, wed_sales#20 AS wed_sales1#32, thu_sales#21 AS thu_sales1#33, fri_sales#22 AS fri_sales1#34, sat_sales#23 AS sat_sales1#35] -(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] +(22) ReusedExchange [Reuses operator id: 9] +Output [9]: [d_week_seq#5, ss_store_sk#1, sum#36, sum#37, sum#38, sum#39, sum#40, sum#41, sum#42] -(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] - -(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] +(23) CometHashAggregate +Input [9]: [d_week_seq#5, ss_store_sk#1, sum#36, sum#37, sum#38, sum#39, sum#40, sum#41, sum#42] 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))#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] -(27) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#50, s_store_id#51] +(24) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#43, s_store_id#44] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_id)] ReadSchema: struct -(28) CometFilter -Input [2]: [s_store_sk#50, s_store_id#51] -Condition : (isnotnull(s_store_sk#50) AND isnotnull(s_store_id#51)) - -(29) ColumnarToRow [codegen id : 5] -Input [2]: [s_store_sk#50, s_store_id#51] +(25) CometFilter +Input [2]: [s_store_sk#43, s_store_id#44] +Condition : (isnotnull(s_store_sk#43) AND isnotnull(s_store_id#44)) -(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] +(26) CometBroadcastExchange +Input [2]: [s_store_sk#43, s_store_id#44] +Arguments: [s_store_sk#43, s_store_id#44] -(31) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#50] -Join type: Inner -Join condition: None +(27) CometBroadcastHashJoin +Left output [9]: [d_week_seq#5, ss_store_sk#1, sun_sales#17, mon_sales#18, tue_sales#19, wed_sales#20, thu_sales#21, fri_sales#22, sat_sales#23] +Right output [2]: [s_store_sk#43, s_store_id#44] +Arguments: [ss_store_sk#1], [s_store_sk#43], Inner, BuildRight -(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] +(28) CometProject +Input [11]: [d_week_seq#5, ss_store_sk#1, sun_sales#17, mon_sales#18, tue_sales#19, wed_sales#20, thu_sales#21, fri_sales#22, sat_sales#23, s_store_sk#43, s_store_id#44] +Arguments: [d_week_seq#5, sun_sales#17, mon_sales#18, tue_sales#19, wed_sales#20, thu_sales#21, fri_sales#22, sat_sales#23, s_store_id#44], [d_week_seq#5, sun_sales#17, mon_sales#18, tue_sales#19, wed_sales#20, thu_sales#21, fri_sales#22, sat_sales#23, s_store_id#44] -(33) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#52, d_week_seq#53] +(29) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_month_seq#45, d_week_seq#46] 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 -(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)) +(30) CometFilter +Input [2]: [d_month_seq#45, d_week_seq#46] +Condition : (((isnotnull(d_month_seq#45) AND (d_month_seq#45 >= 1224)) AND (d_month_seq#45 <= 1235)) AND isnotnull(d_week_seq#46)) -(35) CometProject -Input [2]: [d_month_seq#52, d_week_seq#53] -Arguments: [d_week_seq#53], [d_week_seq#53] +(31) CometProject +Input [2]: [d_month_seq#45, d_week_seq#46] +Arguments: [d_week_seq#46], [d_week_seq#46] -(36) ColumnarToRow [codegen id : 6] -Input [1]: [d_week_seq#53] +(32) CometBroadcastExchange +Input [1]: [d_week_seq#46] +Arguments: [d_week_seq#46] -(37) BroadcastExchange -Input [1]: [d_week_seq#53] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(33) CometBroadcastHashJoin +Left output [9]: [d_week_seq#5, sun_sales#17, mon_sales#18, tue_sales#19, wed_sales#20, thu_sales#21, fri_sales#22, sat_sales#23, s_store_id#44] +Right output [1]: [d_week_seq#46] +Arguments: [d_week_seq#5], [d_week_seq#46], Inner, BuildRight -(38) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [d_week_seq#5] -Right keys [1]: [d_week_seq#53] -Join type: Inner -Join condition: None +(34) CometProject +Input [10]: [d_week_seq#5, sun_sales#17, mon_sales#18, tue_sales#19, wed_sales#20, thu_sales#21, fri_sales#22, sat_sales#23, s_store_id#44, d_week_seq#46] +Arguments: [d_week_seq2#47, s_store_id2#48, sun_sales2#49, mon_sales2#50, tue_sales2#51, wed_sales2#52, thu_sales2#53, fri_sales2#54, sat_sales2#55], [d_week_seq#5 AS d_week_seq2#47, s_store_id#44 AS s_store_id2#48, sun_sales#17 AS sun_sales2#49, mon_sales#18 AS mon_sales2#50, tue_sales#19 AS tue_sales2#51, wed_sales#20 AS wed_sales2#52, thu_sales#21 AS thu_sales2#53, fri_sales#22 AS fri_sales2#54, sat_sales#23 AS sat_sales2#55] -(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] +(35) CometBroadcastExchange +Input [9]: [d_week_seq2#47, s_store_id2#48, sun_sales2#49, mon_sales2#50, tue_sales2#51, wed_sales2#52, thu_sales2#53, fri_sales2#54, sat_sales2#55] +Arguments: [d_week_seq2#47, s_store_id2#48, sun_sales2#49, mon_sales2#50, tue_sales2#51, wed_sales2#52, thu_sales2#53, fri_sales2#54, sat_sales2#55] -(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] +(36) CometBroadcastHashJoin +Left output [10]: [s_store_name1#26, d_week_seq1#27, s_store_id1#28, sun_sales1#29, mon_sales1#30, tue_sales1#31, wed_sales1#32, thu_sales1#33, fri_sales1#34, sat_sales1#35] +Right output [9]: [d_week_seq2#47, s_store_id2#48, sun_sales2#49, mon_sales2#50, tue_sales2#51, wed_sales2#52, thu_sales2#53, fri_sales2#54, sat_sales2#55] +Arguments: [s_store_id1#28, d_week_seq1#27], [s_store_id2#48, (d_week_seq2#47 - 52)], Inner, BuildRight -(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 +(37) CometProject +Input [19]: [s_store_name1#26, d_week_seq1#27, s_store_id1#28, sun_sales1#29, mon_sales1#30, tue_sales1#31, wed_sales1#32, thu_sales1#33, fri_sales1#34, sat_sales1#35, d_week_seq2#47, s_store_id2#48, sun_sales2#49, mon_sales2#50, tue_sales2#51, wed_sales2#52, thu_sales2#53, fri_sales2#54, sat_sales2#55] +Arguments: [s_store_name1#26, s_store_id1#28, d_week_seq1#27, (sun_sales1 / sun_sales2)#56, (mon_sales1 / mon_sales2)#57, (tue_sales1 / tue_sales2)#58, (wed_sales1 / wed_sales2)#59, (thu_sales1 / thu_sales2)#60, (fri_sales1 / fri_sales2)#61, (sat_sales1 / sat_sales2)#62], [s_store_name1#26, s_store_id1#28, d_week_seq1#27, (sun_sales1#29 / sun_sales2#49) AS (sun_sales1 / sun_sales2)#56, (mon_sales1#30 / mon_sales2#50) AS (mon_sales1 / mon_sales2)#57, (tue_sales1#31 / tue_sales2#51) AS (tue_sales1 / tue_sales2)#58, (wed_sales1#32 / wed_sales2#52) AS (wed_sales1 / wed_sales2)#59, (thu_sales1#33 / thu_sales2#53) AS (thu_sales1 / thu_sales2)#60, (fri_sales1#34 / fri_sales2#54) AS (fri_sales1 / fri_sales2)#61, (sat_sales1#35 / sat_sales2#55) AS (sat_sales1 / sat_sales2)#62] -(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] +(38) CometTakeOrderedAndProject +Input [10]: [s_store_name1#26, s_store_id1#28, d_week_seq1#27, (sun_sales1 / sun_sales2)#56, (mon_sales1 / mon_sales2)#57, (tue_sales1 / tue_sales2)#58, (wed_sales1 / wed_sales2)#59, (thu_sales1 / thu_sales2)#60, (fri_sales1 / fri_sales2)#61, (sat_sales1 / sat_sales2)#62] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name1#26 ASC NULLS FIRST,s_store_id1#28 ASC NULLS FIRST,d_week_seq1#27 ASC NULLS FIRST], output=[s_store_name1#26,s_store_id1#28,d_week_seq1#27,(sun_sales1 / sun_sales2)#56,(mon_sales1 / mon_sales2)#57,(tue_sales1 / tue_sales2)#58,(wed_sales1 / wed_sales2)#59,(thu_sales1 / thu_sales2)#60,(fri_sales1 / fri_sales2)#61,(sat_sales1 / sat_sales2)#62]), [s_store_name1#26, s_store_id1#28, d_week_seq1#27, (sun_sales1 / sun_sales2)#56, (mon_sales1 / mon_sales2)#57, (tue_sales1 / tue_sales2)#58, (wed_sales1 / wed_sales2)#59, (thu_sales1 / thu_sales2)#60, (fri_sales1 / fri_sales2)#61, (sat_sales1 / sat_sales2)#62], 100, [s_store_name1#26 ASC NULLS FIRST, s_store_id1#28 ASC NULLS FIRST, d_week_seq1#27 ASC NULLS FIRST], [s_store_name1#26, s_store_id1#28, d_week_seq1#27, (sun_sales1 / sun_sales2)#56, (mon_sales1 / mon_sales2)#57, (tue_sales1 / tue_sales2)#58, (wed_sales1 / wed_sales2)#59, (thu_sales1 / thu_sales2)#60, (fri_sales1 / fri_sales2)#61, (sat_sales1 / sat_sales2)#62] -(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] +(39) ColumnarToRow [codegen id : 1] +Input [10]: [s_store_name1#26, s_store_id1#28, d_week_seq1#27, (sun_sales1 / sun_sales2)#56, (mon_sales1 / mon_sales2)#57, (tue_sales1 / tue_sales2)#58, (wed_sales1 / wed_sales2)#59, (thu_sales1 / thu_sales2)#60, (fri_sales1 / fri_sales2)#61, (sat_sales1 / sat_sales2)#62] 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 6c1e6cd92..bb7a85817 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,62 +1,41 @@ -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 (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] - 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,s_store_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum,d_day_name,ss_sales_price] - CometProject [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] - CometBroadcastHashJoin [ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_week_seq,d_day_name] - CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #2 - CometFilter [d_date_sk,d_week_seq,d_day_name] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_id,s_store_name] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_month_seq,d_week_seq] - CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] - InputAdapter - BroadcastExchange #5 - 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] - BroadcastHashJoin [ss_store_sk,s_store_sk] - 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 - ReusedExchange [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] #1 - InputAdapter - BroadcastExchange #6 - 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 (6) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_month_seq,d_week_seq] - CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [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)] + CometProject [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] [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)] + CometBroadcastHashJoin [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] + CometProject [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] + CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name,d_week_seq] + CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] + CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id,s_store_name] + CometHashAggregate [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,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))] + CometColumnarExchange [d_week_seq,ss_store_sk] #1 + CometHashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum,d_day_name,ss_sales_price] + CometProject [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] + CometBroadcastHashJoin [ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_week_seq,d_day_name] + CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #2 + CometFilter [d_date_sk,d_week_seq,d_day_name] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 + CometFilter [s_store_sk,s_store_id,s_store_name] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] + CometBroadcastExchange [d_week_seq] #4 + CometProject [d_week_seq] + CometFilter [d_month_seq,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] + CometBroadcastExchange [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 + CometProject [d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] + CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,d_week_seq] + CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] + CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id] + CometHashAggregate [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,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))] + ReusedExchange [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] #1 + CometBroadcastExchange [s_store_sk,s_store_id] #6 + CometFilter [s_store_sk,s_store_id] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange [d_week_seq] #7 + CometProject [d_week_seq] + CometFilter [d_month_seq,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_month_seq,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 95a19de48..0341c20d7 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,44 +1,42 @@ == Physical Plan == -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) +* ColumnarToRow (38) ++- CometTakeOrderedAndProject (37) + +- CometFilter (36) + +- CometHashAggregate (35) + +- CometColumnarExchange (34) + +- CometHashAggregate (33) + +- CometProject (32) + +- CometBroadcastHashJoin (31) + :- 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) + +- CometBroadcastExchange (30) + +- CometProject (29) + +- CometBroadcastHashJoin (28) + :- CometFilter (20) + : +- CometScan parquet spark_catalog.default.item (19) + +- CometBroadcastExchange (27) + +- CometFilter (26) + +- CometHashAggregate (25) + +- CometColumnarExchange (24) + +- CometHashAggregate (23) + +- CometFilter (22) + +- CometScan parquet spark_catalog.default.item (21) (1) Scan parquet spark_catalog.default.customer_address @@ -129,186 +127,170 @@ Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight 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] - -(20) Scan parquet spark_catalog.default.item +(19) 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 -(21) CometFilter +(20) 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)) -(22) ColumnarToRow [codegen id : 3] -Input [3]: [i_item_sk#13, i_current_price#14, i_category#15] - -(23) Scan parquet spark_catalog.default.item +(21) 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 -(24) CometFilter +(22) CometFilter Input [2]: [i_current_price#16, i_category#17] Condition : isnotnull(i_category#17) -(25) CometHashAggregate +(23) CometHashAggregate Input [2]: [i_current_price#16, i_category#17] Keys [1]: [i_category#17] Functions [1]: [partial_avg(UnscaledValue(i_current_price#16))] -(26) ColumnarToRow [codegen id : 1] +(24) CometColumnarExchange Input [3]: [i_category#17, sum#18, count#19] +Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(27) Exchange -Input [3]: [i_category#17, sum#18, count#19] -Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, [plan_id=1] - -(28) HashAggregate [codegen id : 2] +(25) CometHashAggregate 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] -(29) Filter [codegen id : 2] -Input [2]: [avg(i_current_price)#21, i_category#17] -Condition : isnotnull(avg(i_current_price)#21) +(26) CometFilter +Input [2]: [avg(i_current_price)#20, i_category#17] +Condition : isnotnull(avg(i_current_price)#20) -(30) BroadcastExchange -Input [2]: [avg(i_current_price)#21, i_category#17] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=2] +(27) CometBroadcastExchange +Input [2]: [avg(i_current_price)#20, i_category#17] +Arguments: [avg(i_current_price)#20, i_category#17] -(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#14 as decimal(14,7)) > (1.2 * avg(i_current_price)#21)) +(28) CometBroadcastHashJoin +Left output [3]: [i_item_sk#13, i_current_price#14, i_category#15] +Right output [2]: [avg(i_current_price)#20, i_category#17] +Arguments: [i_category#15], [i_category#17], Inner, (cast(i_current_price#14 as decimal(14,7)) > (1.2 * avg(i_current_price)#20)), BuildRight -(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] +(29) CometProject +Input [5]: [i_item_sk#13, i_current_price#14, i_category#15, avg(i_current_price)#20, i_category#17] +Arguments: [i_item_sk#13], [i_item_sk#13] -(33) BroadcastExchange +(30) CometBroadcastExchange Input [1]: [i_item_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: [i_item_sk#13] -(34) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#5] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None +(31) CometBroadcastHashJoin +Left output [2]: [ca_state#2, ss_item_sk#5] +Right output [1]: [i_item_sk#13] +Arguments: [ss_item_sk#5], [i_item_sk#13], Inner, BuildRight -(35) Project [codegen id : 4] -Output [1]: [ca_state#2] +(32) CometProject Input [3]: [ca_state#2, ss_item_sk#5, i_item_sk#13] +Arguments: [ca_state#2], [ca_state#2] -(36) HashAggregate [codegen id : 4] +(33) CometHashAggregate Input [1]: [ca_state#2] Keys [1]: [ca_state#2] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#22] -Results [2]: [ca_state#2, count#23] -(37) Exchange -Input [2]: [ca_state#2, count#23] -Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(34) CometColumnarExchange +Input [2]: [ca_state#2, count#21] +Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(38) HashAggregate [codegen id : 5] -Input [2]: [ca_state#2, count#23] +(35) CometHashAggregate +Input [2]: [ca_state#2, count#21] Keys [1]: [ca_state#2] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#24] -Results [2]: [ca_state#2 AS state#25, count(1)#24 AS cnt#26] -(39) Filter [codegen id : 5] -Input [2]: [state#25, cnt#26] -Condition : (cnt#26 >= 10) +(36) CometFilter +Input [2]: [state#22, cnt#23] +Condition : (cnt#23 >= 10) -(40) TakeOrderedAndProject -Input [2]: [state#25, cnt#26] -Arguments: 100, [cnt#26 ASC NULLS FIRST], [state#25, cnt#26] +(37) CometTakeOrderedAndProject +Input [2]: [state#22, cnt#23] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[cnt#23 ASC NULLS FIRST], output=[state#22,cnt#23]), [state#22, cnt#23], 100, [cnt#23 ASC NULLS FIRST], [state#22, cnt#23] + +(38) ColumnarToRow [codegen id : 1] +Input [2]: [state#22, cnt#23] ===== Subqueries ===== 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) +BroadcastExchange (43) ++- * ColumnarToRow (42) + +- CometProject (41) + +- CometFilter (40) + +- CometScan parquet spark_catalog.default.date_dim (39) -(41) Scan parquet spark_catalog.default.date_dim +(39) 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 -(42) CometFilter +(40) 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)) -(43) CometProject +(41) CometProject Input [2]: [d_date_sk#9, d_month_seq#10] Arguments: [d_date_sk#9], [d_date_sk#9] -(44) ColumnarToRow [codegen id : 1] +(42) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(45) BroadcastExchange +(43) BroadcastExchange Input [1]: [d_date_sk#9] -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 = 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) +Subquery:2 Hosting operator id = 40 Hosting Expression = Subquery scalar-subquery#11, [id=#12] +* ColumnarToRow (50) ++- CometHashAggregate (49) + +- CometColumnarExchange (48) + +- CometHashAggregate (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan parquet spark_catalog.default.date_dim (44) -(46) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#27, d_year#28, d_moy#29] +(44) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_month_seq#24, d_year#25, d_moy#26] 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 -(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)) +(45) CometFilter +Input [3]: [d_month_seq#24, d_year#25, d_moy#26] +Condition : (((isnotnull(d_year#25) AND isnotnull(d_moy#26)) AND (d_year#25 = 2000)) AND (d_moy#26 = 1)) -(48) CometProject -Input [3]: [d_month_seq#27, d_year#28, d_moy#29] -Arguments: [d_month_seq#27], [d_month_seq#27] +(46) CometProject +Input [3]: [d_month_seq#24, d_year#25, d_moy#26] +Arguments: [d_month_seq#24], [d_month_seq#24] -(49) CometHashAggregate -Input [1]: [d_month_seq#27] -Keys [1]: [d_month_seq#27] +(47) CometHashAggregate +Input [1]: [d_month_seq#24] +Keys [1]: [d_month_seq#24] Functions: [] -(50) ColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#27] - -(51) Exchange -Input [1]: [d_month_seq#27] -Arguments: hashpartitioning(d_month_seq#27, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(48) CometColumnarExchange +Input [1]: [d_month_seq#24] +Arguments: hashpartitioning(d_month_seq#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(52) HashAggregate [codegen id : 2] -Input [1]: [d_month_seq#27] -Keys [1]: [d_month_seq#27] +(49) CometHashAggregate +Input [1]: [d_month_seq#24] +Keys [1]: [d_month_seq#24] Functions: [] -Aggregate Attributes: [] -Results [1]: [d_month_seq#27] + +(50) ColumnarToRow [codegen id : 1] +Input [1]: [d_month_seq#24] 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 824670101..0e983b96a 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,73 +1,59 @@ -TakeOrderedAndProject [cnt,state] - WholeStageCodegen (5) - Filter [cnt] - HashAggregate [ca_state,count] [count(1),state,cnt,count] - InputAdapter - Exchange [ca_state] #1 - WholeStageCodegen (4) - HashAggregate [ca_state] [count,count] - Project [ca_state] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometProject [ca_state,ss_item_sk] - CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] - CometProject [ca_state,ss_item_sk,ss_sold_date_sk] - CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] - CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_customer_sk,ss_sold_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 #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [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_month_seq,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 [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [d_month_seq] #2 - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - Project [i_item_sk] - BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_current_price,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - InputAdapter - 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] #9 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_category,sum,count,i_current_price] - CometFilter [i_current_price,i_category] - CometScan parquet spark_catalog.default.item [i_current_price,i_category] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [state,cnt] + CometFilter [state,cnt] + CometHashAggregate [state,cnt,ca_state,count,count(1)] + CometColumnarExchange [ca_state] #1 + CometHashAggregate [ca_state,count] + CometProject [ca_state] + CometBroadcastHashJoin [ca_state,ss_item_sk,i_item_sk] + CometProject [ca_state,ss_item_sk] + CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] + CometProject [ca_state,ss_item_sk,ss_sold_date_sk] + CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] + CometProject [ca_state,c_customer_sk] + CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_customer_sk,ss_sold_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 #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + Subquery #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [d_month_seq] + CometColumnarExchange [d_month_seq] #5 + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_month_seq,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 [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + ReusedSubquery [d_month_seq] #2 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk] #7 + CometProject [i_item_sk] + CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] + CometFilter [i_item_sk,i_current_price,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] + CometBroadcastExchange [avg(i_current_price),i_category] #8 + CometFilter [avg(i_current_price),i_category] + CometHashAggregate [avg(i_current_price),i_category,sum,count,avg(UnscaledValue(i_current_price))] + CometColumnarExchange [i_category] #9 + CometHashAggregate [i_category,sum,count,i_current_price] + CometFilter [i_current_price,i_category] + CometScan parquet spark_catalog.default.item [i_current_price,i_category] 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 0f61456dd..0aeb4e2b4 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 @@ -1,67 +1,65 @@ == Physical Plan == -TakeOrderedAndProject (63) -+- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- Union (59) - :- * HashAggregate (28) - : +- Exchange (27) - : +- * 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) - : +- * 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) - +- * 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) +* ColumnarToRow (61) ++- CometTakeOrderedAndProject (60) + +- CometHashAggregate (59) + +- CometColumnarExchange (58) + +- CometHashAggregate (57) + +- CometUnion (56) + :- CometHashAggregate (27) + : +- CometColumnarExchange (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) + :- CometHashAggregate (41) + : +- CometColumnarExchange (40) + : +- CometHashAggregate (39) + : +- CometProject (38) + : +- CometBroadcastHashJoin (37) + : :- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometProject (32) + : : : +- CometBroadcastHashJoin (31) + : : : :- CometFilter (29) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (28) + : : : +- ReusedExchange (30) + : : +- ReusedExchange (33) + : +- ReusedExchange (36) + +- CometHashAggregate (55) + +- CometColumnarExchange (54) + +- CometHashAggregate (53) + +- CometProject (52) + +- CometBroadcastHashJoin (51) + :- CometProject (49) + : +- CometBroadcastHashJoin (48) + : :- CometProject (46) + : : +- CometBroadcastHashJoin (45) + : : :- CometFilter (43) + : : : +- CometScan parquet spark_catalog.default.web_sales (42) + : : +- ReusedExchange (44) + : +- ReusedExchange (47) + +- ReusedExchange (50) (1) Scan parquet spark_catalog.default.store_sales @@ -185,212 +183,199 @@ 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))] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarExchange Input [2]: [i_item_id#12, sum#15] +Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(27) Exchange -Input [2]: [i_item_id#12, sum#15] -Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=1] - -(28) HashAggregate [codegen id : 2] +(27) CometHashAggregate 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))#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#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] +(28) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_bill_addr_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#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#22)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#20)] PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(30) CometFilter -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)) +(29) CometFilter +Input [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] +Condition : (isnotnull(cs_bill_addr_sk#16) AND isnotnull(cs_item_sk#17)) -(31) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#23] +(30) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#21] -(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 +(31) CometBroadcastHashJoin +Left output [4]: [cs_bill_addr_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 -(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] +(32) CometProject +Input [5]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19, d_date_sk#21] +Arguments: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18], [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18] -(34) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#24] +(33) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#22] -(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 +(34) CometBroadcastHashJoin +Left output [3]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18] +Right output [1]: [ca_address_sk#22] +Arguments: [cs_bill_addr_sk#16], [ca_address_sk#22], 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] +(35) CometProject +Input [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, ca_address_sk#22] +Arguments: [cs_item_sk#17, cs_ext_sales_price#18], [cs_item_sk#17, cs_ext_sales_price#18] -(37) ReusedExchange [Reuses operator id: 22] -Output [2]: [i_item_sk#25, i_item_id#26] +(36) ReusedExchange [Reuses operator id: 22] +Output [2]: [i_item_sk#23, i_item_id#24] -(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 +(37) CometBroadcastHashJoin +Left output [2]: [cs_item_sk#17, cs_ext_sales_price#18] +Right output [2]: [i_item_sk#23, i_item_id#24] +Arguments: [cs_item_sk#17], [i_item_sk#23], 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] +(38) CometProject +Input [4]: [cs_item_sk#17, cs_ext_sales_price#18, i_item_sk#23, i_item_id#24] +Arguments: [cs_ext_sales_price#18, i_item_id#24], [cs_ext_sales_price#18, i_item_id#24] -(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))] +(39) CometHashAggregate +Input [2]: [cs_ext_sales_price#18, i_item_id#24] +Keys [1]: [i_item_id#24] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#18))] -(41) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_id#26, sum#27] +(40) CometColumnarExchange +Input [2]: [i_item_id#24, sum#25] +Arguments: hashpartitioning(i_item_id#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(42) Exchange -Input [2]: [i_item_id#26, sum#27] -Arguments: hashpartitioning(i_item_id#26, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(41) CometHashAggregate +Input [2]: [i_item_id#24, sum#25] +Keys [1]: [i_item_id#24] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#18))] -(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] +(42) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] 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#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#30)] 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)) +(43) CometFilter +Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] +Condition : (isnotnull(ws_bill_addr_sk#27) AND isnotnull(ws_item_sk#26)) -(46) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#35] +(44) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#31] -(47) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] -Right output [1]: [d_date_sk#35] -Arguments: [ws_sold_date_sk#33], [d_date_sk#35], Inner, BuildRight +(45) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] +Right output [1]: [d_date_sk#31] +Arguments: [ws_sold_date_sk#29], [d_date_sk#31], Inner, BuildRight -(48) CometProject -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] -Arguments: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32], [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] +(46) CometProject +Input [5]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29, d_date_sk#31] +Arguments: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28], [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28] -(49) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#36] +(47) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#32] -(50) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] -Right output [1]: [ca_address_sk#36] -Arguments: [ws_bill_addr_sk#31], [ca_address_sk#36], Inner, BuildRight +(48) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28] +Right output [1]: [ca_address_sk#32] +Arguments: [ws_bill_addr_sk#27], [ca_address_sk#32], Inner, BuildRight -(51) CometProject -Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ca_address_sk#36] -Arguments: [ws_item_sk#30, ws_ext_sales_price#32], [ws_item_sk#30, ws_ext_sales_price#32] +(49) CometProject +Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ca_address_sk#32] +Arguments: [ws_item_sk#26, ws_ext_sales_price#28], [ws_item_sk#26, ws_ext_sales_price#28] -(52) ReusedExchange [Reuses operator id: 22] -Output [2]: [i_item_sk#37, i_item_id#38] +(50) ReusedExchange [Reuses operator id: 22] +Output [2]: [i_item_sk#33, i_item_id#34] -(53) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_ext_sales_price#32] -Right output [2]: [i_item_sk#37, i_item_id#38] -Arguments: [ws_item_sk#30], [i_item_sk#37], Inner, BuildRight +(51) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#26, ws_ext_sales_price#28] +Right output [2]: [i_item_sk#33, i_item_id#34] +Arguments: [ws_item_sk#26], [i_item_sk#33], Inner, BuildRight -(54) CometProject -Input [4]: [ws_item_sk#30, ws_ext_sales_price#32, i_item_sk#37, i_item_id#38] -Arguments: [ws_ext_sales_price#32, i_item_id#38], [ws_ext_sales_price#32, i_item_id#38] +(52) CometProject +Input [4]: [ws_item_sk#26, ws_ext_sales_price#28, i_item_sk#33, i_item_id#34] +Arguments: [ws_ext_sales_price#28, i_item_id#34], [ws_ext_sales_price#28, i_item_id#34] -(55) CometHashAggregate -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))] +(53) CometHashAggregate +Input [2]: [ws_ext_sales_price#28, i_item_id#34] +Keys [1]: [i_item_id#34] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#28))] -(56) ColumnarToRow [codegen id : 5] -Input [2]: [i_item_id#38, sum#39] +(54) CometColumnarExchange +Input [2]: [i_item_id#34, sum#35] +Arguments: hashpartitioning(i_item_id#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(57) Exchange -Input [2]: [i_item_id#38, sum#39] -Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(58) HashAggregate [codegen id : 6] -Input [2]: [i_item_id#38, sum#39] -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))#40] -Results [2]: [i_item_id#38, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#32))#40,17,2) AS total_sales#41] +(55) CometHashAggregate +Input [2]: [i_item_id#34, sum#35] +Keys [1]: [i_item_id#34] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#28))] -(59) Union +(56) CometUnion +Child 0 Input [2]: [i_item_id#12, total_sales#36] +Child 1 Input [2]: [i_item_id#24, total_sales#37] +Child 2 Input [2]: [i_item_id#34, total_sales#38] -(60) HashAggregate [codegen id : 7] -Input [2]: [i_item_id#12, total_sales#17] +(57) CometHashAggregate +Input [2]: [i_item_id#12, total_sales#36] Keys [1]: [i_item_id#12] -Functions [1]: [partial_sum(total_sales#17)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [3]: [i_item_id#12, sum#44, isEmpty#45] +Functions [1]: [partial_sum(total_sales#36)] -(61) Exchange -Input [3]: [i_item_id#12, sum#44, isEmpty#45] -Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(58) CometColumnarExchange +Input [3]: [i_item_id#12, sum#39, isEmpty#40] +Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(62) HashAggregate [codegen id : 8] -Input [3]: [i_item_id#12, sum#44, isEmpty#45] +(59) CometHashAggregate +Input [3]: [i_item_id#12, sum#39, isEmpty#40] Keys [1]: [i_item_id#12] -Functions [1]: [sum(total_sales#17)] -Aggregate Attributes [1]: [sum(total_sales#17)#46] -Results [2]: [i_item_id#12, sum(total_sales#17)#46 AS total_sales#47] +Functions [1]: [sum(total_sales#36)] + +(60) CometTakeOrderedAndProject +Input [2]: [i_item_id#12, total_sales#41] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#12 ASC NULLS FIRST,total_sales#41 ASC NULLS FIRST], output=[i_item_id#12,total_sales#41]), [i_item_id#12, total_sales#41], 100, [i_item_id#12 ASC NULLS FIRST, total_sales#41 ASC NULLS FIRST], [i_item_id#12, total_sales#41] -(63) TakeOrderedAndProject -Input [2]: [i_item_id#12, total_sales#47] -Arguments: 100, [i_item_id#12 ASC NULLS FIRST, total_sales#47 ASC NULLS FIRST], [i_item_id#12, total_sales#47] +(61) ColumnarToRow [codegen id : 1] +Input [2]: [i_item_id#12, total_sales#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (68) -+- * ColumnarToRow (67) - +- CometProject (66) - +- CometFilter (65) - +- CometScan parquet spark_catalog.default.date_dim (64) +BroadcastExchange (66) ++- * ColumnarToRow (65) + +- CometProject (64) + +- CometFilter (63) + +- CometScan parquet spark_catalog.default.date_dim (62) -(64) Scan parquet spark_catalog.default.date_dim +(62) Scan parquet spark_catalog.default.date_dim 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 +(63) 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)) -(66) CometProject +(64) CometProject 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] +(65) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(68) BroadcastExchange +(66) BroadcastExchange Input [1]: [d_date_sk#6] 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#21 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 28 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#29 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 7cfcb75da..871f477b7 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,89 +1,73 @@ -TakeOrderedAndProject [i_item_id,total_sales] - WholeStageCodegen (8) - HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - InputAdapter - Exchange [i_item_id] #1 - WholeStageCodegen (7) - HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (2) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_item_id] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,sum,ss_ext_sales_price] - CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,total_sales] + CometHashAggregate [i_item_id,total_sales,sum,isEmpty,sum(total_sales)] + CometColumnarExchange [i_item_id] #1 + CometHashAggregate [i_item_id,sum,isEmpty,total_sales] + CometUnion [i_item_id,total_sales] + CometHashAggregate [i_item_id,total_sales,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange [i_item_id] #2 + CometHashAggregate [i_item_id,sum,ss_ext_sales_price] + CometProject [ss_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ca_address_sk] #5 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [i_item_id] #7 - CometProject [i_item_id] - CometFilter [i_item_id,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] #8 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,sum,cs_ext_sales_price] - CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] - CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] - CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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 - 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] #9 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,sum,ws_ext_sales_price] - CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] - CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] - CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ca_address_sk] #5 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange [i_item_id] #7 + CometProject [i_item_id] + CometFilter [i_item_id,i_category] + CometScan parquet spark_catalog.default.item [i_item_id,i_category] + CometHashAggregate [i_item_id,total_sales,sum,sum(UnscaledValue(cs_ext_sales_price))] + CometColumnarExchange [i_item_id] #8 + CometHashAggregate [i_item_id,sum,cs_ext_sales_price] + CometProject [cs_ext_sales_price,i_item_id] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] + CometProject [cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] + CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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 + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #6 + CometHashAggregate [i_item_id,total_sales,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometColumnarExchange [i_item_id] #9 + CometHashAggregate [i_item_id,sum,ws_ext_sales_price] + CometProject [ws_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] + CometProject [ws_item_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] + CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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 + 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 1567198fe..3bbacfbb9 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,9 +1,9 @@ == Physical Plan == * Project (65) +- * BroadcastNestedLoopJoin Inner BuildRight (64) - :- * HashAggregate (41) - : +- Exchange (40) - : +- * ColumnarToRow (39) + :- * ColumnarToRow (41) + : +- CometHashAggregate (40) + : +- CometColumnarExchange (39) : +- CometHashAggregate (38) : +- CometProject (37) : +- CometBroadcastHashJoin (36) @@ -43,9 +43,9 @@ : +- CometFilter (33) : +- CometScan parquet spark_catalog.default.item (32) +- BroadcastExchange (63) - +- * HashAggregate (62) - +- Exchange (61) - +- * ColumnarToRow (60) + +- * ColumnarToRow (62) + +- CometHashAggregate (61) + +- CometColumnarExchange (60) +- CometHashAggregate (59) +- CometProject (58) +- CometBroadcastHashJoin (57) @@ -247,122 +247,118 @@ Input [1]: [ss_ext_sales_price#5] Keys: [] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] -(39) ColumnarToRow [codegen id : 1] +(39) CometColumnarExchange Input [1]: [sum#23] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(40) Exchange -Input [1]: [sum#23] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] - -(41) HashAggregate [codegen id : 4] +(40) CometHashAggregate Input [1]: [sum#23] Keys: [] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -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] + +(41) ColumnarToRow [codegen id : 2] +Input [1]: [promotions#24] (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] +Output [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#30), dynamicpruningexpression(ss_sold_date_sk#30 IN dynamicpruning#31)] +PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#30)] PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (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)) +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)) (44) ReusedExchange [Reuses operator id: 6] -Output [1]: [s_store_sk#32] +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 +Left output [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] +Right output [1]: [s_store_sk#31] +Arguments: [ss_store_sk#27], [s_store_sk#31], Inner, BuildRight (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] +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] +Arguments: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29], [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29] (47) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#33] +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 +Left output [4]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29] +Right output [1]: [d_date_sk#32] +Arguments: [ss_sold_date_sk#29], [d_date_sk#32], Inner, BuildRight (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] +Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#32] +Arguments: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28], [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28] (50) ReusedExchange [Reuses operator id: 23] -Output [2]: [c_customer_sk#34, c_current_addr_sk#35] +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 +Left output [3]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28] +Right output [2]: [c_customer_sk#33, c_current_addr_sk#34] +Arguments: [ss_customer_sk#26], [c_customer_sk#33], Inner, BuildRight (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] +Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, c_customer_sk#33, c_current_addr_sk#34] +Arguments: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34], [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34] (53) ReusedExchange [Reuses operator id: 29] -Output [1]: [ca_address_sk#36] +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 +Left output [3]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34] +Right output [1]: [ca_address_sk#35] +Arguments: [c_current_addr_sk#34], [ca_address_sk#35], Inner, BuildRight (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] +Input [4]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34, ca_address_sk#35] +Arguments: [ss_item_sk#25, ss_ext_sales_price#28], [ss_item_sk#25, ss_ext_sales_price#28] (56) ReusedExchange [Reuses operator id: 35] -Output [1]: [i_item_sk#37] +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 +Left output [2]: [ss_item_sk#25, ss_ext_sales_price#28] +Right output [1]: [i_item_sk#36] +Arguments: [ss_item_sk#25], [i_item_sk#36], Inner, BuildRight (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] +Input [3]: [ss_item_sk#25, ss_ext_sales_price#28, i_item_sk#36] +Arguments: [ss_ext_sales_price#28], [ss_ext_sales_price#28] (59) CometHashAggregate -Input [1]: [ss_ext_sales_price#29] +Input [1]: [ss_ext_sales_price#28] Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#29))] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#28))] -(60) ColumnarToRow [codegen id : 2] -Input [1]: [sum#38] +(60) CometColumnarExchange +Input [1]: [sum#37] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(61) Exchange -Input [1]: [sum#38] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] - -(62) HashAggregate [codegen id : 3] -Input [1]: [sum#38] +(61) CometHashAggregate +Input [1]: [sum#37] Keys: [] -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] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#28))] + +(62) ColumnarToRow [codegen id : 1] +Input [1]: [total#38] (63) BroadcastExchange -Input [1]: [total#40] +Input [1]: [total#38] Arguments: IdentityBroadcastMode, [plan_id=3] -(64) BroadcastNestedLoopJoin [codegen id : 4] +(64) BroadcastNestedLoopJoin [codegen id : 2] Join type: Inner Join condition: None -(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] +(65) Project [codegen id : 2] +Output [3]: [promotions#24, total#38, ((cast(promotions#24 as decimal(15,4)) / cast(total#38 as decimal(15,4))) * 100) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#39] +Input [2]: [promotions#24, total#38] ===== Subqueries ===== @@ -396,6 +392,6 @@ Input [1]: [d_date_sk#14] Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -Subquery:2 Hosting operator id = 42 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 42 Hosting Expression = ss_sold_date_sk#29 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 1b2af33b0..b95b0d616 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,83 +1,79 @@ -WholeStageCodegen (4) +WholeStageCodegen (2) Project [promotions,total] BroadcastNestedLoopJoin - HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),promotions,sum] + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [sum,ss_ext_sales_price] - CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] - CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk] - CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk] - CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,p_promo_sk] - CometProject [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk] #3 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_gmt_offset] - CometScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] - CometBroadcastExchange [p_promo_sk] #4 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #6 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk] #7 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk] #8 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_category] + CometHashAggregate [promotions,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange #1 + CometHashAggregate [sum,ss_ext_sales_price] + CometProject [ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk] + CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,p_promo_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk] #3 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_gmt_offset] + CometScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] + CometBroadcastExchange [p_promo_sk] #4 + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] + CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #6 + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange [ca_address_sk] #7 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometBroadcastExchange [i_item_sk] #8 + CometProject [i_item_sk] + CometFilter [i_item_sk,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_category] InputAdapter BroadcastExchange #9 - WholeStageCodegen (3) - HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),total,sum] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange #10 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometHashAggregate [sum,ss_ext_sales_price] - CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] - CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk] - CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk] - CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_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 - 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 + CometHashAggregate [total,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange #10 + CometHashAggregate [sum,ss_ext_sales_price] + CometProject [ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk] + CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_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 + 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 21d44db2c..f58411101 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,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (28) -+- * HashAggregate (27) - +- Exchange (26) - +- * ColumnarToRow (25) +* ColumnarToRow (28) ++- CometTakeOrderedAndProject (27) + +- CometHashAggregate (26) + +- CometColumnarExchange (25) +- CometHashAggregate (24) +- CometProject (23) +- CometBroadcastHashJoin (22) @@ -145,21 +145,19 @@ Input [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#9, web_name#11, _group 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)] -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(27) HashAggregate [codegen id : 2] +(26) CometHashAggregate 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)#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] -(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] +(27) CometTakeOrderedAndProject +Input [8]: [substr(w_warehouse_name, 1, 20)#20, sm_type#9, web_name#11, 30 days #21, 31 - 60 days #22, 61 - 90 days #23, 91 - 120 days #24, >120 days #25] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#20 ASC NULLS FIRST,sm_type#9 ASC NULLS FIRST,web_name#11 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#20,sm_type#9,web_name#11,30 days #21,31 - 60 days #22,61 - 90 days #23,91 - 120 days #24,>120 days #25]), [substr(w_warehouse_name, 1, 20)#20, sm_type#9, web_name#11, 30 days #21, 31 - 60 days #22, 61 - 90 days #23, 91 - 120 days #24, >120 days #25], 100, [substr(w_warehouse_name, 1, 20)#20 ASC NULLS FIRST, sm_type#9 ASC NULLS FIRST, web_name#11 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#20, sm_type#9, web_name#11, 30 days #21, 31 - 60 days #22, 61 - 90 days #23, 91 - 120 days #24, >120 days #25] + +(28) ColumnarToRow [codegen id : 1] +Input [8]: [substr(w_warehouse_name, 1, 20)#20, sm_type#9, web_name#11, 30 days #21, 31 - 60 days #22, 61 - 90 days #23, 91 - 120 days #24, >120 days #25] 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 0b4fc61a1..c7bcf72f0 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,32 +1,30 @@ -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 (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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum,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,ws_sold_date_sk,w_warehouse_name,sm_type,web_name,d_date_sk] - CometProject [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name] - CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_site_sk,web_name] - CometProject [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type] - CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] - CometProject [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name] - CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name] - CometFilter [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_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] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 - CometFilter [sm_ship_mode_sk,sm_type] - CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange [web_site_sk,web_name] #4 - CometFilter [web_site_sk,web_name] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_name] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] + CometHashAggregate [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,_groupingexpression,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)] + CometColumnarExchange [_groupingexpression,sm_type,web_name] #1 + CometHashAggregate [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum,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,ws_sold_date_sk,w_warehouse_name,sm_type,web_name,d_date_sk] + CometProject [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_site_sk,web_name] + CometProject [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] + CometProject [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_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] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 + CometFilter [w_warehouse_sk,w_warehouse_name] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 + CometFilter [sm_ship_mode_sk,sm_type] + CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] + CometBroadcastExchange [web_site_sk,web_name] #4 + CometFilter [web_site_sk,web_name] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_name] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + 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 621e9c8c6..5fd263660 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 @@ -3,11 +3,11 @@ TakeOrderedAndProject (29) +- * Project (28) +- * Filter (27) +- Window (26) - +- * Sort (25) - +- Exchange (24) - +- * HashAggregate (23) - +- Exchange (22) - +- * ColumnarToRow (21) + +- * ColumnarToRow (25) + +- CometSort (24) + +- CometColumnarExchange (23) + +- CometHashAggregate (22) + +- CometColumnarExchange (21) +- CometHashAggregate (20) +- CometProject (19) +- CometBroadcastHashJoin (18) @@ -127,43 +127,41 @@ 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))] -(21) ColumnarToRow [codegen id : 1] +(21) CometColumnarExchange Input [3]: [i_manager_id#5, d_moy#17, sum#19] +Arguments: hashpartitioning(i_manager_id#5, d_moy#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(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] - -(23) HashAggregate [codegen id : 2] +(22) CometHashAggregate 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] -(24) Exchange -Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] -Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(23) CometColumnarExchange +Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] +Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(24) CometSort +Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] +Arguments: [i_manager_id#5, sum_sales#20, _w0#21], [i_manager_id#5 ASC NULLS FIRST] -(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) ColumnarToRow [codegen id : 1] +Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] (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] +Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] +Arguments: [avg(_w0#21) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_manager_id#5] -(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) Filter [codegen id : 2] +Input [4]: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22] +Condition : CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) ELSE false END -(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) Project [codegen id : 2] +Output [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] +Input [4]: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22] (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] +Input [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] +Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST], [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] ===== Subqueries ===== 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 b2033c7a9..1b538c9ab 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,45 +1,41 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] - WholeStageCodegen (4) + WholeStageCodegen (2) Project [i_manager_id,sum_sales,avg_monthly_sales] Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_manager_id] - WholeStageCodegen (3) - Sort [i_manager_id] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [i_manager_id] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_manager_id,d_moy,sum,ss_sales_price] - CometProject [i_manager_id,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] - CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] - CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_item_sk,i_manager_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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_date_sk,d_month_seq,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] + CometSort [i_manager_id,sum_sales,_w0] + CometColumnarExchange [i_manager_id] #1 + CometHashAggregate [i_manager_id,sum_sales,_w0,d_moy,sum,sum(UnscaledValue(ss_sales_price))] + CometColumnarExchange [i_manager_id,d_moy] #2 + CometHashAggregate [i_manager_id,d_moy,sum,ss_sales_price] + CometProject [i_manager_id,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] + CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] + CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_item_sk,i_manager_id] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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_date_sk,d_month_seq,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometBroadcastExchange [d_date_sk,d_moy] #5 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_month_seq,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometBroadcastExchange [s_store_sk] #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 bd491e60f..b4e05a864 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 @@ -1,185 +1,180 @@ == Physical Plan == -* Sort (181) -+- Exchange (180) - +- * Project (179) - +- * SortMergeJoin Inner (178) - :- * Sort (110) - : +- Exchange (109) - : +- * HashAggregate (108) - : +- * HashAggregate (107) - : +- * Project (106) - : +- * BroadcastHashJoin Inner BuildRight (105) - : :- * Project (99) - : : +- * BroadcastHashJoin Inner BuildRight (98) - : : :- * Project (96) - : : : +- * BroadcastHashJoin Inner BuildRight (95) - : : : :- * Project (90) - : : : : +- * BroadcastHashJoin Inner BuildRight (89) - : : : : :- * Project (87) - : : : : : +- * BroadcastHashJoin Inner BuildRight (86) - : : : : : :- * Project (81) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (80) - : : : : : : :- * Project (78) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (77) - : : : : : : : :- * Project (72) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (71) - : : : : : : : : :- * Project (66) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (65) - : : : : : : : : : :- * Project (63) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (62) - : : : : : : : : : : :- * Project (57) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (56) - : : : : : : : : : : : :- * Project (54) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (53) - : : : : : : : : : : : : :- * Project (48) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (47) - : : : : : : : : : : : : : :- * Project (42) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : : : : : : : : : : : :- * Project (36) - : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (35) - : : : : : : : : : : : : : : : :- * Project (33) - : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (32) - : : : : : : : : : : : : : : : : :- * Sort (11) - : : : : : : : : : : : : : : : : : +- Exchange (10) - : : : : : : : : : : : : : : : : : +- * ColumnarToRow (9) - : : : : : : : : : : : : : : : : : +- CometProject (8) - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) - : : : : : : : : : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- CometProject (6) - : : : : : : : : : : : : : : : : : +- CometFilter (5) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : : : : : : : : : : : : : : : +- * Sort (31) - : : : : : : : : : : : : : : : : +- * Project (30) - : : : : : : : : : : : : : : : : +- * Filter (29) - : : : : : : : : : : : : : : : : +- * HashAggregate (28) - : : : : : : : : : : : : : : : : +- Exchange (27) - : : : : : : : : : : : : : : : : +- * HashAggregate (26) - : : : : : : : : : : : : : : : : +- * Project (25) - : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (24) - : : : : : : : : : : : : : : : : :- * Sort (17) - : : : : : : : : : : : : : : : : : +- Exchange (16) - : : : : : : : : : : : : : : : : : +- * ColumnarToRow (15) - : : : : : : : : : : : : : : : : : +- CometProject (14) - : : : : : : : : : : : : : : : : : +- CometFilter (13) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (12) - : : : : : : : : : : : : : : : : +- * Sort (23) - : : : : : : : : : : : : : : : : +- Exchange (22) - : : : : : : : : : : : : : : : : +- * ColumnarToRow (21) - : : : : : : : : : : : : : : : : +- CometProject (20) - : : : : : : : : : : : : : : : : +- CometFilter (19) - : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (18) - : : : : : : : : : : : : : : : +- ReusedExchange (34) - : : : : : : : : : : : : : : +- BroadcastExchange (40) - : : : : : : : : : : : : : : +- * ColumnarToRow (39) - : : : : : : : : : : : : : : +- CometFilter (38) - : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (37) - : : : : : : : : : : : : : +- BroadcastExchange (46) - : : : : : : : : : : : : : +- * ColumnarToRow (45) - : : : : : : : : : : : : : +- CometFilter (44) - : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (43) - : : : : : : : : : : : : +- BroadcastExchange (52) - : : : : : : : : : : : : +- * ColumnarToRow (51) - : : : : : : : : : : : : +- CometFilter (50) - : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (49) - : : : : : : : : : : : +- ReusedExchange (55) - : : : : : : : : : : +- BroadcastExchange (61) - : : : : : : : : : : +- * ColumnarToRow (60) - : : : : : : : : : : +- CometFilter (59) - : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (58) - : : : : : : : : : +- ReusedExchange (64) - : : : : : : : : +- BroadcastExchange (70) - : : : : : : : : +- * ColumnarToRow (69) - : : : : : : : : +- CometFilter (68) - : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (67) - : : : : : : : +- BroadcastExchange (76) - : : : : : : : +- * ColumnarToRow (75) - : : : : : : : +- CometFilter (74) - : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (73) - : : : : : : +- ReusedExchange (79) - : : : : : +- BroadcastExchange (85) - : : : : : +- * ColumnarToRow (84) - : : : : : +- CometFilter (83) - : : : : : +- CometScan parquet spark_catalog.default.customer_address (82) - : : : : +- ReusedExchange (88) - : : : +- BroadcastExchange (94) - : : : +- * ColumnarToRow (93) - : : : +- CometFilter (92) - : : : +- CometScan parquet spark_catalog.default.income_band (91) - : : +- ReusedExchange (97) - : +- BroadcastExchange (104) - : +- * ColumnarToRow (103) - : +- CometProject (102) - : +- CometFilter (101) - : +- CometScan parquet spark_catalog.default.item (100) - +- * Sort (177) - +- Exchange (176) - +- * HashAggregate (175) - +- * HashAggregate (174) - +- * Project (173) - +- * BroadcastHashJoin Inner BuildRight (172) - :- * Project (170) - : +- * BroadcastHashJoin Inner BuildRight (169) - : :- * Project (167) - : : +- * BroadcastHashJoin Inner BuildRight (166) - : : :- * Project (164) - : : : +- * BroadcastHashJoin Inner BuildRight (163) - : : : :- * Project (161) - : : : : +- * BroadcastHashJoin Inner BuildRight (160) - : : : : :- * Project (158) - : : : : : +- * BroadcastHashJoin Inner BuildRight (157) - : : : : : :- * Project (155) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (154) - : : : : : : :- * Project (152) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (151) - : : : : : : : :- * Project (149) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (148) - : : : : : : : : :- * Project (146) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (145) - : : : : : : : : : :- * Project (143) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (142) - : : : : : : : : : : :- * Project (140) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (139) - : : : : : : : : : : : :- * Project (137) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (136) - : : : : : : : : : : : : :- * Project (134) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (133) - : : : : : : : : : : : : : :- * Project (131) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (130) - : : : : : : : : : : : : : : :- * Project (128) - : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (127) - : : : : : : : : : : : : : : : :- * Sort (121) - : : : : : : : : : : : : : : : : +- Exchange (120) - : : : : : : : : : : : : : : : : +- * ColumnarToRow (119) - : : : : : : : : : : : : : : : : +- CometProject (118) - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (117) - : : : : : : : : : : : : : : : : :- CometBroadcastExchange (113) - : : : : : : : : : : : : : : : : : +- CometFilter (112) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (111) - : : : : : : : : : : : : : : : : +- CometProject (116) - : : : : : : : : : : : : : : : : +- CometFilter (115) - : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (114) - : : : : : : : : : : : : : : : +- * Sort (126) - : : : : : : : : : : : : : : : +- * Project (125) - : : : : : : : : : : : : : : : +- * Filter (124) - : : : : : : : : : : : : : : : +- * HashAggregate (123) - : : : : : : : : : : : : : : : +- ReusedExchange (122) - : : : : : : : : : : : : : : +- ReusedExchange (129) - : : : : : : : : : : : : : +- ReusedExchange (132) - : : : : : : : : : : : : +- ReusedExchange (135) - : : : : : : : : : : : +- ReusedExchange (138) - : : : : : : : : : : +- ReusedExchange (141) - : : : : : : : : : +- ReusedExchange (144) - : : : : : : : : +- ReusedExchange (147) - : : : : : : : +- ReusedExchange (150) - : : : : : : +- ReusedExchange (153) - : : : : : +- ReusedExchange (156) - : : : : +- ReusedExchange (159) - : : : +- ReusedExchange (162) - : : +- ReusedExchange (165) - : +- ReusedExchange (168) - +- ReusedExchange (171) +* ColumnarToRow (176) ++- CometSort (175) + +- CometColumnarExchange (174) + +- RowToColumnar (173) + +- * Project (172) + +- * SortMergeJoin Inner (171) + :- * ColumnarToRow (101) + : +- CometSort (100) + : +- CometColumnarExchange (99) + : +- CometHashAggregate (98) + : +- CometHashAggregate (97) + : +- CometProject (96) + : +- CometBroadcastHashJoin (95) + : :- CometProject (90) + : : +- CometBroadcastHashJoin (89) + : : :- CometProject (87) + : : : +- CometBroadcastHashJoin (86) + : : : :- CometProject (82) + : : : : +- CometBroadcastHashJoin (81) + : : : : :- CometProject (79) + : : : : : +- CometBroadcastHashJoin (78) + : : : : : :- CometProject (74) + : : : : : : +- CometBroadcastHashJoin (73) + : : : : : : :- CometProject (71) + : : : : : : : +- CometBroadcastHashJoin (70) + : : : : : : : :- CometProject (66) + : : : : : : : : +- CometBroadcastHashJoin (65) + : : : : : : : : :- CometProject (61) + : : : : : : : : : +- CometBroadcastHashJoin (60) + : : : : : : : : : :- CometProject (58) + : : : : : : : : : : +- CometBroadcastHashJoin (57) + : : : : : : : : : : :- CometProject (53) + : : : : : : : : : : : +- CometBroadcastHashJoin (52) + : : : : : : : : : : : :- CometProject (50) + : : : : : : : : : : : : +- CometBroadcastHashJoin (49) + : : : : : : : : : : : : :- CometProject (45) + : : : : : : : : : : : : : +- CometBroadcastHashJoin (44) + : : : : : : : : : : : : : :- CometProject (40) + : : : : : : : : : : : : : : +- CometBroadcastHashJoin (39) + : : : : : : : : : : : : : : :- CometProject (35) + : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (34) + : : : : : : : : : : : : : : : :- CometProject (30) + : : : : : : : : : : : : : : : : +- CometSortMergeJoin (29) + : : : : : : : : : : : : : : : : :- CometSort (10) + : : : : : : : : : : : : : : : : : +- CometColumnarExchange (9) + : : : : : : : : : : : : : : : : : +- CometProject (8) + : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) + : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) + : : : : : : : : : : : : : : : : : : +- CometFilter (2) + : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : : : : : : : : : : : : +- CometProject (6) + : : : : : : : : : : : : : : : : : +- CometFilter (5) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : : : : : : : : : : : : : : : +- CometSort (28) + : : : : : : : : : : : : : : : : +- CometProject (27) + : : : : : : : : : : : : : : : : +- CometFilter (26) + : : : : : : : : : : : : : : : : +- CometHashAggregate (25) + : : : : : : : : : : : : : : : : +- CometColumnarExchange (24) + : : : : : : : : : : : : : : : : +- CometHashAggregate (23) + : : : : : : : : : : : : : : : : +- CometProject (22) + : : : : : : : : : : : : : : : : +- CometSortMergeJoin (21) + : : : : : : : : : : : : : : : : :- CometSort (15) + : : : : : : : : : : : : : : : : : +- CometColumnarExchange (14) + : : : : : : : : : : : : : : : : : +- CometProject (13) + : : : : : : : : : : : : : : : : : +- CometFilter (12) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (11) + : : : : : : : : : : : : : : : : +- CometSort (20) + : : : : : : : : : : : : : : : : +- CometColumnarExchange (19) + : : : : : : : : : : : : : : : : +- CometProject (18) + : : : : : : : : : : : : : : : : +- CometFilter (17) + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (16) + : : : : : : : : : : : : : : : +- CometBroadcastExchange (33) + : : : : : : : : : : : : : : : +- CometFilter (32) + : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (31) + : : : : : : : : : : : : : : +- CometBroadcastExchange (38) + : : : : : : : : : : : : : : +- CometFilter (37) + : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (36) + : : : : : : : : : : : : : +- CometBroadcastExchange (43) + : : : : : : : : : : : : : +- CometFilter (42) + : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (41) + : : : : : : : : : : : : +- CometBroadcastExchange (48) + : : : : : : : : : : : : +- CometFilter (47) + : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (46) + : : : : : : : : : : : +- ReusedExchange (51) + : : : : : : : : : : +- CometBroadcastExchange (56) + : : : : : : : : : : +- CometFilter (55) + : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (54) + : : : : : : : : : +- ReusedExchange (59) + : : : : : : : : +- CometBroadcastExchange (64) + : : : : : : : : +- CometFilter (63) + : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (62) + : : : : : : : +- CometBroadcastExchange (69) + : : : : : : : +- CometFilter (68) + : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (67) + : : : : : : +- ReusedExchange (72) + : : : : : +- CometBroadcastExchange (77) + : : : : : +- CometFilter (76) + : : : : : +- CometScan parquet spark_catalog.default.customer_address (75) + : : : : +- ReusedExchange (80) + : : : +- CometBroadcastExchange (85) + : : : +- CometFilter (84) + : : : +- CometScan parquet spark_catalog.default.income_band (83) + : : +- ReusedExchange (88) + : +- CometBroadcastExchange (94) + : +- CometProject (93) + : +- CometFilter (92) + : +- CometScan parquet spark_catalog.default.item (91) + +- * ColumnarToRow (170) + +- CometSort (169) + +- CometColumnarExchange (168) + +- CometHashAggregate (167) + +- CometHashAggregate (166) + +- CometProject (165) + +- CometBroadcastHashJoin (164) + :- CometProject (162) + : +- CometBroadcastHashJoin (161) + : :- CometProject (159) + : : +- CometBroadcastHashJoin (158) + : : :- CometProject (156) + : : : +- CometBroadcastHashJoin (155) + : : : :- CometProject (153) + : : : : +- CometBroadcastHashJoin (152) + : : : : :- CometProject (150) + : : : : : +- CometBroadcastHashJoin (149) + : : : : : :- CometProject (147) + : : : : : : +- CometBroadcastHashJoin (146) + : : : : : : :- CometProject (144) + : : : : : : : +- CometBroadcastHashJoin (143) + : : : : : : : :- CometProject (141) + : : : : : : : : +- CometBroadcastHashJoin (140) + : : : : : : : : :- CometProject (138) + : : : : : : : : : +- CometBroadcastHashJoin (137) + : : : : : : : : : :- CometProject (135) + : : : : : : : : : : +- CometBroadcastHashJoin (134) + : : : : : : : : : : :- CometProject (132) + : : : : : : : : : : : +- CometBroadcastHashJoin (131) + : : : : : : : : : : : :- CometProject (129) + : : : : : : : : : : : : +- CometBroadcastHashJoin (128) + : : : : : : : : : : : : :- CometProject (126) + : : : : : : : : : : : : : +- CometBroadcastHashJoin (125) + : : : : : : : : : : : : : :- CometProject (123) + : : : : : : : : : : : : : : +- CometBroadcastHashJoin (122) + : : : : : : : : : : : : : : :- CometProject (118) + : : : : : : : : : : : : : : : +- CometSortMergeJoin (117) + : : : : : : : : : : : : : : : :- CometSort (111) + : : : : : : : : : : : : : : : : +- CometColumnarExchange (110) + : : : : : : : : : : : : : : : : +- CometProject (109) + : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (108) + : : : : : : : : : : : : : : : : :- CometBroadcastExchange (104) + : : : : : : : : : : : : : : : : : +- CometFilter (103) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (102) + : : : : : : : : : : : : : : : : +- CometProject (107) + : : : : : : : : : : : : : : : : +- CometFilter (106) + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (105) + : : : : : : : : : : : : : : : +- CometSort (116) + : : : : : : : : : : : : : : : +- CometProject (115) + : : : : : : : : : : : : : : : +- CometFilter (114) + : : : : : : : : : : : : : : : +- CometHashAggregate (113) + : : : : : : : : : : : : : : : +- ReusedExchange (112) + : : : : : : : : : : : : : : +- CometBroadcastExchange (121) + : : : : : : : : : : : : : : +- CometFilter (120) + : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (119) + : : : : : : : : : : : : : +- ReusedExchange (124) + : : : : : : : : : : : : +- ReusedExchange (127) + : : : : : : : : : : : +- ReusedExchange (130) + : : : : : : : : : : +- ReusedExchange (133) + : : : : : : : : : +- ReusedExchange (136) + : : : : : : : : +- ReusedExchange (139) + : : : : : : : +- ReusedExchange (142) + : : : : : : +- ReusedExchange (145) + : : : : : +- ReusedExchange (148) + : : : : +- ReusedExchange (151) + : : : +- ReusedExchange (154) + : : +- ReusedExchange (157) + : +- ReusedExchange (160) + +- ReusedExchange (163) (1) Scan parquet spark_catalog.default.store_sales @@ -222,843 +217,793 @@ Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number# 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] Arguments: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -(9) ColumnarToRow [codegen id : 1] +(9) CometColumnarExchange Input [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(10) Exchange +(10) CometSort Input [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1 ASC NULLS FIRST] -(11) Sort [codegen id : 2] -Input [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 - -(12) Scan parquet spark_catalog.default.catalog_sales +(11) Scan parquet spark_catalog.default.catalog_sales Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_order_number)] ReadSchema: struct -(13) CometFilter +(12) CometFilter Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) -(14) CometProject +(13) CometProject Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -(15) ColumnarToRow [codegen id : 3] -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] - -(16) Exchange +(14) CometColumnarExchange Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(17) Sort [codegen id : 4] +(15) CometSort Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST], false, 0 +Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST] -(18) Scan parquet spark_catalog.default.catalog_returns +(16) Scan parquet spark_catalog.default.catalog_returns Output [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(19) CometFilter +(17) CometFilter Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) -(20) CometProject +(18) CometProject Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -(21) ColumnarToRow [codegen id : 5] +(19) CometColumnarExchange Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) Exchange +(20) CometSort Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST] -(23) Sort [codegen id : 6] -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST], false, 0 +(21) CometSortMergeJoin +Left output [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] +Right output [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number#22], Inner -(24) SortMergeJoin [codegen id : 7] -Left keys [2]: [cs_item_sk#17, cs_order_number#18] -Right keys [2]: [cr_item_sk#21, cr_order_number#22] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 7] -Output [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +(22) CometProject Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +Arguments: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -(26) HashAggregate [codegen id : 7] +(23) CometHashAggregate Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Keys [1]: [cs_item_sk#17] Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] -Aggregate Attributes [3]: [sum#27, sum#28, isEmpty#29] -Results [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] -(27) Exchange -Input [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(24) CometColumnarExchange +Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] +Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(28) HashAggregate [codegen id : 8] -Input [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] +(25) CometHashAggregate +Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] Keys [1]: [cs_item_sk#17] Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#19))#33, sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))#34] -Results [3]: [cs_item_sk#17, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#19))#33,17,2) AS sale#35, sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))#34 AS refund#36] -(29) Filter [codegen id : 8] -Input [3]: [cs_item_sk#17, sale#35, refund#36] -Condition : ((isnotnull(sale#35) AND isnotnull(refund#36)) AND (cast(sale#35 as decimal(21,2)) > (2 * refund#36))) +(26) CometFilter +Input [3]: [cs_item_sk#17, sale#30, refund#31] +Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) -(30) Project [codegen id : 8] -Output [1]: [cs_item_sk#17] -Input [3]: [cs_item_sk#17, sale#35, refund#36] +(27) CometProject +Input [3]: [cs_item_sk#17, sale#30, refund#31] +Arguments: [cs_item_sk#17], [cs_item_sk#17] -(31) Sort [codegen id : 8] +(28) CometSort Input [1]: [cs_item_sk#17] -Arguments: [cs_item_sk#17 ASC NULLS FIRST], false, 0 +Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] -(32) SortMergeJoin [codegen id : 24] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [cs_item_sk#17] -Join type: Inner -Join condition: None +(29) CometSortMergeJoin +Left output [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Right output [1]: [cs_item_sk#17] +Arguments: [ss_item_sk#1], [cs_item_sk#17], Inner -(33) Project [codegen id : 24] -Output [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +(30) CometProject Input [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#17] +Arguments: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] + +(31) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#32, d_year#33] +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 -(34) ReusedExchange [Reuses operator id: 185] -Output [2]: [d_date_sk#37, d_year#38] +(32) CometFilter +Input [2]: [d_date_sk#32, d_year#33] +Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(35) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#37] -Join type: Inner -Join condition: None +(33) CometBroadcastExchange +Input [2]: [d_date_sk#32, d_year#33] +Arguments: [d_date_sk#32, d_year#33] + +(34) CometBroadcastHashJoin +Left output [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Right output [2]: [d_date_sk#32, d_year#33] +Arguments: [ss_sold_date_sk#12], [d_date_sk#32], Inner, BuildRight -(36) Project [codegen id : 24] -Output [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38] -Input [13]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#37, d_year#38] +(35) CometProject +Input [13]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#32, d_year#33] +Arguments: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33], [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] -(37) Scan parquet spark_catalog.default.store -Output [3]: [s_store_sk#39, s_store_name#40, s_zip#41] +(36) Scan parquet spark_catalog.default.store +Output [3]: [s_store_sk#34, s_store_name#35, s_zip#36] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_zip)] ReadSchema: struct -(38) CometFilter -Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] -Condition : ((isnotnull(s_store_sk#39) AND isnotnull(s_store_name#40)) AND isnotnull(s_zip#41)) +(37) CometFilter +Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] +Condition : ((isnotnull(s_store_sk#34) AND isnotnull(s_store_name#35)) AND isnotnull(s_zip#36)) -(39) ColumnarToRow [codegen id : 10] -Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] +(38) CometBroadcastExchange +Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] +Arguments: [s_store_sk#34, s_store_name#35, s_zip#36] -(40) BroadcastExchange -Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +(39) CometBroadcastHashJoin +Left output [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] +Right output [3]: [s_store_sk#34, s_store_name#35, s_zip#36] +Arguments: [ss_store_sk#6], [s_store_sk#34], Inner, BuildRight -(41) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_store_sk#6] -Right keys [1]: [s_store_sk#39] -Join type: Inner -Join condition: None +(40) 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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_sk#34, s_store_name#35, s_zip#36] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36] -(42) Project [codegen id : 24] -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41] -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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_sk#39, s_store_name#40, s_zip#41] - -(43) Scan parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] +(41) Scan parquet spark_catalog.default.customer +Output [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(44) CometFilter -Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] -Condition : (((((isnotnull(c_customer_sk#42) AND isnotnull(c_first_sales_date_sk#47)) AND isnotnull(c_first_shipto_date_sk#46)) AND isnotnull(c_current_cdemo_sk#43)) AND isnotnull(c_current_hdemo_sk#44)) AND isnotnull(c_current_addr_sk#45)) - -(45) ColumnarToRow [codegen id : 11] -Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] +(42) CometFilter +Input [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +Condition : (((((isnotnull(c_customer_sk#37) AND isnotnull(c_first_sales_date_sk#42)) AND isnotnull(c_first_shipto_date_sk#41)) AND isnotnull(c_current_cdemo_sk#38)) AND isnotnull(c_current_hdemo_sk#39)) AND isnotnull(c_current_addr_sk#40)) -(46) BroadcastExchange -Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +(43) CometBroadcastExchange +Input [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +Arguments: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] -(47) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#42] -Join type: Inner -Join condition: None +(44) 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_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36] +Right output [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +Arguments: [ss_customer_sk#2], [c_customer_sk#37], Inner, BuildRight -(48) Project [codegen id : 24] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] +(45) CometProject +Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] -(49) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#48, d_year#49] +(46) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#43, d_year#44] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct -(50) CometFilter -Input [2]: [d_date_sk#48, d_year#49] -Condition : isnotnull(d_date_sk#48) +(47) CometFilter +Input [2]: [d_date_sk#43, d_year#44] +Condition : isnotnull(d_date_sk#43) -(51) ColumnarToRow [codegen id : 12] -Input [2]: [d_date_sk#48, d_year#49] +(48) CometBroadcastExchange +Input [2]: [d_date_sk#43, d_year#44] +Arguments: [d_date_sk#43, d_year#44] -(52) BroadcastExchange -Input [2]: [d_date_sk#48, d_year#49] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] - -(53) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [c_first_sales_date_sk#47] -Right keys [1]: [d_date_sk#48] -Join type: Inner -Join condition: None +(49) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +Right output [2]: [d_date_sk#43, d_year#44] +Arguments: [c_first_sales_date_sk#42], [d_date_sk#43], Inner, BuildRight -(54) Project [codegen id : 24] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, d_year#49] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47, d_date_sk#48, d_year#49] +(50) CometProject +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42, d_date_sk#43, d_year#44] +Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, d_year#44], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, d_year#44] -(55) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#50, d_year#51] +(51) ReusedExchange [Reuses operator id: 48] +Output [2]: [d_date_sk#45, d_year#46] -(56) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [c_first_shipto_date_sk#46] -Right keys [1]: [d_date_sk#50] -Join type: Inner -Join condition: None +(52) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, d_year#44] +Right output [2]: [d_date_sk#45, d_year#46] +Arguments: [c_first_shipto_date_sk#41], [d_date_sk#45], Inner, BuildRight -(57) Project [codegen id : 24] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, d_year#49, d_date_sk#50, d_year#51] +(53) CometProject +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, d_year#44, d_date_sk#45, d_year#46] +Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] -(58) Scan parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#52, cd_marital_status#53] +(54) Scan parquet spark_catalog.default.customer_demographics +Output [2]: [cd_demo_sk#47, cd_marital_status#48] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status)] ReadSchema: struct -(59) CometFilter -Input [2]: [cd_demo_sk#52, cd_marital_status#53] -Condition : (isnotnull(cd_demo_sk#52) AND isnotnull(cd_marital_status#53)) - -(60) ColumnarToRow [codegen id : 14] -Input [2]: [cd_demo_sk#52, cd_marital_status#53] +(55) CometFilter +Input [2]: [cd_demo_sk#47, cd_marital_status#48] +Condition : (isnotnull(cd_demo_sk#47) AND isnotnull(cd_marital_status#48)) -(61) BroadcastExchange -Input [2]: [cd_demo_sk#52, cd_marital_status#53] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +(56) CometBroadcastExchange +Input [2]: [cd_demo_sk#47, cd_marital_status#48] +Arguments: [cd_demo_sk#47, cd_marital_status#48] -(62) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#52] -Join type: Inner -Join condition: None +(57) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] +Right output [2]: [cd_demo_sk#47, cd_marital_status#48] +Arguments: [ss_cdemo_sk#3], [cd_demo_sk#47], Inner, BuildRight -(63) Project [codegen id : 24] -Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_marital_status#53] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_demo_sk#52, cd_marital_status#53] +(58) CometProject +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_demo_sk#47, cd_marital_status#48] +Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_marital_status#48], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_marital_status#48] -(64) ReusedExchange [Reuses operator id: 61] -Output [2]: [cd_demo_sk#54, cd_marital_status#55] +(59) ReusedExchange [Reuses operator id: 56] +Output [2]: [cd_demo_sk#49, cd_marital_status#50] -(65) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [c_current_cdemo_sk#43] -Right keys [1]: [cd_demo_sk#54] -Join type: Inner -Join condition: NOT (cd_marital_status#53 = cd_marital_status#55) +(60) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_marital_status#48] +Right output [2]: [cd_demo_sk#49, cd_marital_status#50] +Arguments: [c_current_cdemo_sk#38], [cd_demo_sk#49], Inner, NOT (cd_marital_status#48 = cd_marital_status#50), BuildRight -(66) Project [codegen id : 24] -Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_marital_status#53, cd_demo_sk#54, cd_marital_status#55] +(61) CometProject +Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_marital_status#48, cd_demo_sk#49, cd_marital_status#50] +Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] -(67) Scan parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#56] +(62) Scan parquet spark_catalog.default.promotion +Output [1]: [p_promo_sk#51] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_promo_sk)] ReadSchema: struct -(68) CometFilter -Input [1]: [p_promo_sk#56] -Condition : isnotnull(p_promo_sk#56) +(63) CometFilter +Input [1]: [p_promo_sk#51] +Condition : isnotnull(p_promo_sk#51) -(69) ColumnarToRow [codegen id : 16] -Input [1]: [p_promo_sk#56] +(64) CometBroadcastExchange +Input [1]: [p_promo_sk#51] +Arguments: [p_promo_sk#51] -(70) BroadcastExchange -Input [1]: [p_promo_sk#56] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] +(65) CometBroadcastHashJoin +Left output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] +Right output [1]: [p_promo_sk#51] +Arguments: [ss_promo_sk#7], [p_promo_sk#51], Inner, BuildRight -(71) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_promo_sk#7] -Right keys [1]: [p_promo_sk#56] -Join type: Inner -Join condition: None - -(72) Project [codegen id : 24] -Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, p_promo_sk#56] +(66) CometProject +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, p_promo_sk#51] +Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] -(73) Scan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#57, hd_income_band_sk#58] +(67) Scan parquet spark_catalog.default.household_demographics +Output [2]: [hd_demo_sk#52, hd_income_band_sk#53] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] ReadSchema: struct -(74) CometFilter -Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] -Condition : (isnotnull(hd_demo_sk#57) AND isnotnull(hd_income_band_sk#58)) - -(75) ColumnarToRow [codegen id : 17] -Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] +(68) CometFilter +Input [2]: [hd_demo_sk#52, hd_income_band_sk#53] +Condition : (isnotnull(hd_demo_sk#52) AND isnotnull(hd_income_band_sk#53)) -(76) BroadcastExchange -Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] +(69) CometBroadcastExchange +Input [2]: [hd_demo_sk#52, hd_income_band_sk#53] +Arguments: [hd_demo_sk#52, hd_income_band_sk#53] -(77) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_hdemo_sk#4] -Right keys [1]: [hd_demo_sk#57] -Join type: Inner -Join condition: None +(70) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] +Right output [2]: [hd_demo_sk#52, hd_income_band_sk#53] +Arguments: [ss_hdemo_sk#4], [hd_demo_sk#52], Inner, BuildRight -(78) Project [codegen id : 24] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_demo_sk#57, hd_income_band_sk#58] +(71) CometProject +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_demo_sk#52, hd_income_band_sk#53] +Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53] -(79) ReusedExchange [Reuses operator id: 76] -Output [2]: [hd_demo_sk#59, hd_income_band_sk#60] +(72) ReusedExchange [Reuses operator id: 69] +Output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -(80) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [c_current_hdemo_sk#44] -Right keys [1]: [hd_demo_sk#59] -Join type: Inner -Join condition: None +(73) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53] +Right output [2]: [hd_demo_sk#54, hd_income_band_sk#55] +Arguments: [c_current_hdemo_sk#39], [hd_demo_sk#54], Inner, BuildRight -(81) Project [codegen id : 24] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60] -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_demo_sk#59, hd_income_band_sk#60] +(74) CometProject +Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_demo_sk#54, hd_income_band_sk#55] +Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55] -(82) Scan parquet spark_catalog.default.customer_address -Output [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +(75) Scan parquet spark_catalog.default.customer_address +Output [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(83) CometFilter -Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -Condition : isnotnull(ca_address_sk#61) +(76) CometFilter +Input [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +Condition : isnotnull(ca_address_sk#56) -(84) ColumnarToRow [codegen id : 19] -Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +(77) CometBroadcastExchange +Input [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +Arguments: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] -(85) BroadcastExchange -Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=11] +(78) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55] +Right output [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +Arguments: [ss_addr_sk#5], [ca_address_sk#56], Inner, BuildRight -(86) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_addr_sk#5] -Right keys [1]: [ca_address_sk#61] -Join type: Inner -Join condition: None - -(87) Project [codegen id : 24] -Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +(79) CometProject +Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] -(88) ReusedExchange [Reuses operator id: 85] -Output [5]: [ca_address_sk#66, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] +(80) ReusedExchange [Reuses operator id: 77] +Output [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -(89) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [c_current_addr_sk#45] -Right keys [1]: [ca_address_sk#66] -Join type: Inner -Join condition: None +(81) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +Right output [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Arguments: [c_current_addr_sk#40], [ca_address_sk#61], Inner, BuildRight -(90) Project [codegen id : 24] -Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_address_sk#66, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] +(82) CometProject +Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -(91) Scan parquet spark_catalog.default.income_band -Output [1]: [ib_income_band_sk#71] +(83) Scan parquet spark_catalog.default.income_band +Output [1]: [ib_income_band_sk#66] Batched: true Location [not included in comparison]/{warehouse_dir}/income_band] PushedFilters: [IsNotNull(ib_income_band_sk)] ReadSchema: struct -(92) CometFilter -Input [1]: [ib_income_band_sk#71] -Condition : isnotnull(ib_income_band_sk#71) +(84) CometFilter +Input [1]: [ib_income_band_sk#66] +Condition : isnotnull(ib_income_band_sk#66) -(93) ColumnarToRow [codegen id : 21] -Input [1]: [ib_income_band_sk#71] +(85) CometBroadcastExchange +Input [1]: [ib_income_band_sk#66] +Arguments: [ib_income_band_sk#66] -(94) BroadcastExchange -Input [1]: [ib_income_band_sk#71] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] - -(95) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [hd_income_band_sk#58] -Right keys [1]: [ib_income_band_sk#71] -Join type: Inner -Join condition: None +(86) CometBroadcastHashJoin +Left output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Right output [1]: [ib_income_band_sk#66] +Arguments: [hd_income_band_sk#53], [ib_income_band_sk#66], Inner, BuildRight -(96) Project [codegen id : 24] -Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, ib_income_band_sk#71] +(87) CometProject +Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ib_income_band_sk#66] +Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -(97) ReusedExchange [Reuses operator id: 94] -Output [1]: [ib_income_band_sk#72] +(88) ReusedExchange [Reuses operator id: 85] +Output [1]: [ib_income_band_sk#67] -(98) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [hd_income_band_sk#60] -Right keys [1]: [ib_income_band_sk#72] -Join type: Inner -Join condition: None +(89) CometBroadcastHashJoin +Left output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Right output [1]: [ib_income_band_sk#67] +Arguments: [hd_income_band_sk#55], [ib_income_band_sk#67], Inner, BuildRight -(99) Project [codegen id : 24] -Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, ib_income_band_sk#72] +(90) CometProject +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ib_income_band_sk#67] +Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -(100) Scan parquet spark_catalog.default.item -Output [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] +(91) Scan parquet spark_catalog.default.item +Output [4]: [i_item_sk#68, i_current_price#69, i_color#70, i_product_name#71] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), In(i_color, [burlywood ,floral ,indian ,medium ,purple ,spring ]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] ReadSchema: struct -(101) CometFilter -Input [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] -Condition : ((((((isnotnull(i_current_price#74) AND i_color#75 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#74 >= 64.00)) AND (i_current_price#74 <= 74.00)) AND (i_current_price#74 >= 65.00)) AND (i_current_price#74 <= 79.00)) AND isnotnull(i_item_sk#73)) - -(102) CometProject -Input [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] -Arguments: [i_item_sk#73, i_product_name#76], [i_item_sk#73, i_product_name#76] +(92) CometFilter +Input [4]: [i_item_sk#68, i_current_price#69, i_color#70, i_product_name#71] +Condition : ((((((isnotnull(i_current_price#69) AND i_color#70 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#69 >= 64.00)) AND (i_current_price#69 <= 74.00)) AND (i_current_price#69 >= 65.00)) AND (i_current_price#69 <= 79.00)) AND isnotnull(i_item_sk#68)) -(103) ColumnarToRow [codegen id : 23] -Input [2]: [i_item_sk#73, i_product_name#76] +(93) CometProject +Input [4]: [i_item_sk#68, i_current_price#69, i_color#70, i_product_name#71] +Arguments: [i_item_sk#68, i_product_name#71], [i_item_sk#68, i_product_name#71] -(104) BroadcastExchange -Input [2]: [i_item_sk#73, i_product_name#76] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] +(94) CometBroadcastExchange +Input [2]: [i_item_sk#68, i_product_name#71] +Arguments: [i_item_sk#68, i_product_name#71] -(105) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#73] -Join type: Inner -Join condition: None +(95) CometBroadcastHashJoin +Left output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Right output [2]: [i_item_sk#68, i_product_name#71] +Arguments: [ss_item_sk#1], [i_item_sk#68], Inner, BuildRight -(106) Project [codegen id : 24] -Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, d_year#49, d_year#51, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] +(96) CometProject +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, i_item_sk#68, i_product_name#71] +Arguments: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#44, d_year#46, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, i_item_sk#68, i_product_name#71], [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#44, d_year#46, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, i_item_sk#68, i_product_name#71] -(107) HashAggregate [codegen id : 24] -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, d_year#49, d_year#51, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51] +(97) CometHashAggregate +Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#44, d_year#46, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, i_item_sk#68, i_product_name#71] +Keys [15]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46] Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count#77, sum#78, sum#79, sum#80] -Results [19]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51, count#81, sum#82, sum#83, sum#84] -(108) HashAggregate [codegen id : 24] -Input [19]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51, count#81, sum#82, sum#83, sum#84] -Keys [15]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51] +(98) CometHashAggregate +Input [19]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46, count#72, sum#73, sum#74, sum#75] +Keys [15]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46] Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count(1)#85, sum(UnscaledValue(ss_wholesale_cost#9))#86, sum(UnscaledValue(ss_list_price#10))#87, sum(UnscaledValue(ss_coupon_amt#11))#88] -Results [17]: [i_product_name#76 AS product_name#89, i_item_sk#73 AS item_sk#90, s_store_name#40 AS store_name#91, s_zip#41 AS store_zip#92, ca_street_number#62 AS b_street_number#93, ca_street_name#63 AS b_streen_name#94, ca_city#64 AS b_city#95, ca_zip#65 AS b_zip#96, ca_street_number#67 AS c_street_number#97, ca_street_name#68 AS c_street_name#98, ca_city#69 AS c_city#99, ca_zip#70 AS c_zip#100, d_year#38 AS syear#101, count(1)#85 AS cnt#102, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#86,17,2) AS s1#103, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#87,17,2) AS s2#104, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#88,17,2) AS s3#105] -(109) Exchange -Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] -Arguments: hashpartitioning(item_sk#90, store_name#91, store_zip#92, 5), ENSURE_REQUIREMENTS, [plan_id=14] +(99) CometColumnarExchange +Input [17]: [product_name#76, item_sk#77, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92] +Arguments: hashpartitioning(item_sk#77, store_name#78, store_zip#79, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(100) CometSort +Input [17]: [product_name#76, item_sk#77, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92] +Arguments: [product_name#76, item_sk#77, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92], [item_sk#77 ASC NULLS FIRST, store_name#78 ASC NULLS FIRST, store_zip#79 ASC NULLS FIRST] -(110) Sort [codegen id : 25] -Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] -Arguments: [item_sk#90 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, store_zip#92 ASC NULLS FIRST], false, 0 +(101) ColumnarToRow [codegen id : 1] +Input [17]: [product_name#76, item_sk#77, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92] -(111) Scan parquet spark_catalog.default.store_sales -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] +(102) Scan parquet spark_catalog.default.store_sales +Output [12]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_ticket_number#100, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#117), dynamicpruningexpression(ss_sold_date_sk#117 IN dynamicpruning#118)] +PartitionFilters: [isnotnull(ss_sold_date_sk#104), dynamicpruningexpression(ss_sold_date_sk#104 IN dynamicpruning#105)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct -(112) CometFilter -Input [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] -Condition : (((((((isnotnull(ss_item_sk#106) AND isnotnull(ss_ticket_number#113)) AND isnotnull(ss_store_sk#111)) AND isnotnull(ss_customer_sk#107)) AND isnotnull(ss_cdemo_sk#108)) AND isnotnull(ss_promo_sk#112)) AND isnotnull(ss_hdemo_sk#109)) AND isnotnull(ss_addr_sk#110)) +(103) CometFilter +Input [12]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_ticket_number#100, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] +Condition : (((((((isnotnull(ss_item_sk#93) AND isnotnull(ss_ticket_number#100)) AND isnotnull(ss_store_sk#98)) AND isnotnull(ss_customer_sk#94)) AND isnotnull(ss_cdemo_sk#95)) AND isnotnull(ss_promo_sk#99)) AND isnotnull(ss_hdemo_sk#96)) AND isnotnull(ss_addr_sk#97)) -(113) CometBroadcastExchange -Input [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] -Arguments: [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] +(104) CometBroadcastExchange +Input [12]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_ticket_number#100, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] +Arguments: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_ticket_number#100, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] -(114) Scan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] +(105) Scan parquet spark_catalog.default.store_returns +Output [3]: [sr_item_sk#106, sr_ticket_number#107, sr_returned_date_sk#108] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] ReadSchema: struct -(115) CometFilter -Input [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] -Condition : (isnotnull(sr_item_sk#119) AND isnotnull(sr_ticket_number#120)) +(106) CometFilter +Input [3]: [sr_item_sk#106, sr_ticket_number#107, sr_returned_date_sk#108] +Condition : (isnotnull(sr_item_sk#106) AND isnotnull(sr_ticket_number#107)) -(116) CometProject -Input [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] -Arguments: [sr_item_sk#119, sr_ticket_number#120], [sr_item_sk#119, sr_ticket_number#120] +(107) CometProject +Input [3]: [sr_item_sk#106, sr_ticket_number#107, sr_returned_date_sk#108] +Arguments: [sr_item_sk#106, sr_ticket_number#107], [sr_item_sk#106, sr_ticket_number#107] -(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, BuildLeft +(108) CometBroadcastHashJoin +Left output [12]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_ticket_number#100, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] +Right output [2]: [sr_item_sk#106, sr_ticket_number#107] +Arguments: [ss_item_sk#93, ss_ticket_number#100], [sr_item_sk#106, sr_ticket_number#107], 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] -Arguments: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117], [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +(109) CometProject +Input [14]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_ticket_number#100, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104, sr_item_sk#106, sr_ticket_number#107] +Arguments: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104], [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] -(119) ColumnarToRow [codegen id : 26] -Input [11]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +(110) CometColumnarExchange +Input [11]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] +Arguments: hashpartitioning(ss_item_sk#93, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(120) Exchange -Input [11]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -Arguments: hashpartitioning(ss_item_sk#106, 5), ENSURE_REQUIREMENTS, [plan_id=15] +(111) CometSort +Input [11]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] +Arguments: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104], [ss_item_sk#93 ASC NULLS FIRST] -(121) Sort [codegen id : 27] -Input [11]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -Arguments: [ss_item_sk#106 ASC NULLS FIRST], false, 0 +(112) ReusedExchange [Reuses operator id: 24] +Output [4]: [cs_item_sk#109, sum#110, sum#111, isEmpty#112] -(122) ReusedExchange [Reuses operator id: 27] -Output [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] +(113) CometHashAggregate +Input [4]: [cs_item_sk#109, sum#110, sum#111, isEmpty#112] +Keys [1]: [cs_item_sk#109] +Functions [2]: [sum(UnscaledValue(cs_ext_list_price#113)), sum(((cr_refunded_cash#114 + cr_reversed_charge#115) + cr_store_credit#116))] -(123) HashAggregate [codegen id : 33] -Input [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] -Keys [1]: [cs_item_sk#122] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#126)), sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#126))#33, sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))#34] -Results [3]: [cs_item_sk#122, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#126))#33,17,2) AS sale#35, sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))#34 AS refund#36] +(114) CometFilter +Input [3]: [cs_item_sk#109, sale#30, refund#31] +Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) -(124) Filter [codegen id : 33] -Input [3]: [cs_item_sk#122, sale#35, refund#36] -Condition : ((isnotnull(sale#35) AND isnotnull(refund#36)) AND (cast(sale#35 as decimal(21,2)) > (2 * refund#36))) +(115) CometProject +Input [3]: [cs_item_sk#109, sale#30, refund#31] +Arguments: [cs_item_sk#109], [cs_item_sk#109] -(125) Project [codegen id : 33] -Output [1]: [cs_item_sk#122] -Input [3]: [cs_item_sk#122, sale#35, refund#36] +(116) CometSort +Input [1]: [cs_item_sk#109] +Arguments: [cs_item_sk#109], [cs_item_sk#109 ASC NULLS FIRST] -(126) Sort [codegen id : 33] -Input [1]: [cs_item_sk#122] -Arguments: [cs_item_sk#122 ASC NULLS FIRST], false, 0 +(117) CometSortMergeJoin +Left output [11]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] +Right output [1]: [cs_item_sk#109] +Arguments: [ss_item_sk#93], [cs_item_sk#109], Inner -(127) SortMergeJoin [codegen id : 49] -Left keys [1]: [ss_item_sk#106] -Right keys [1]: [cs_item_sk#122] -Join type: Inner -Join condition: None +(118) CometProject +Input [12]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104, cs_item_sk#109] +Arguments: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104], [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] -(128) Project [codegen id : 49] -Output [11]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -Input [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, cs_item_sk#122] +(119) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#117, d_year#118] +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 -(129) ReusedExchange [Reuses operator id: 189] -Output [2]: [d_date_sk#130, d_year#131] +(120) CometFilter +Input [2]: [d_date_sk#117, d_year#118] +Condition : ((isnotnull(d_year#118) AND (d_year#118 = 2000)) AND isnotnull(d_date_sk#117)) -(130) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [ss_sold_date_sk#117] -Right keys [1]: [d_date_sk#130] -Join type: Inner -Join condition: None +(121) CometBroadcastExchange +Input [2]: [d_date_sk#117, d_year#118] +Arguments: [d_date_sk#117, d_year#118] -(131) Project [codegen id : 49] -Output [11]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131] -Input [13]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, d_date_sk#130, d_year#131] +(122) CometBroadcastHashJoin +Left output [11]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] +Right output [2]: [d_date_sk#117, d_year#118] +Arguments: [ss_sold_date_sk#104], [d_date_sk#117], Inner, BuildRight -(132) ReusedExchange [Reuses operator id: 40] -Output [3]: [s_store_sk#132, s_store_name#133, s_zip#134] +(123) CometProject +Input [13]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104, d_date_sk#117, d_year#118] +Arguments: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118], [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118] -(133) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [ss_store_sk#111] -Right keys [1]: [s_store_sk#132] -Join type: Inner -Join condition: None +(124) ReusedExchange [Reuses operator id: 38] +Output [3]: [s_store_sk#119, s_store_name#120, s_zip#121] -(134) Project [codegen id : 49] -Output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134] -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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_sk#132, s_store_name#133, s_zip#134] +(125) CometBroadcastHashJoin +Left output [11]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118] +Right output [3]: [s_store_sk#119, s_store_name#120, s_zip#121] +Arguments: [ss_store_sk#98], [s_store_sk#119], Inner, BuildRight -(135) ReusedExchange [Reuses operator id: 46] -Output [6]: [c_customer_sk#135, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, c_first_sales_date_sk#140] +(126) CometProject +Input [14]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_sk#119, s_store_name#120, s_zip#121] +Arguments: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121], [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121] -(136) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [ss_customer_sk#107] -Right keys [1]: [c_customer_sk#135] -Join type: Inner -Join condition: None +(127) ReusedExchange [Reuses operator id: 43] +Output [6]: [c_customer_sk#122, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, c_first_sales_date_sk#127] -(137) Project [codegen id : 49] -Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, c_first_sales_date_sk#140] -Input [18]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_customer_sk#135, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, c_first_sales_date_sk#140] +(128) CometBroadcastHashJoin +Left output [12]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121] +Right output [6]: [c_customer_sk#122, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, c_first_sales_date_sk#127] +Arguments: [ss_customer_sk#94], [c_customer_sk#122], Inner, BuildRight -(138) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#141, d_year#142] +(129) CometProject +Input [18]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_customer_sk#122, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, c_first_sales_date_sk#127] +Arguments: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, c_first_sales_date_sk#127], [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, c_first_sales_date_sk#127] -(139) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [c_first_sales_date_sk#140] -Right keys [1]: [d_date_sk#141] -Join type: Inner -Join condition: None +(130) ReusedExchange [Reuses operator id: 48] +Output [2]: [d_date_sk#128, d_year#129] -(140) Project [codegen id : 49] -Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, d_year#142] -Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, c_first_sales_date_sk#140, d_date_sk#141, d_year#142] +(131) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, c_first_sales_date_sk#127] +Right output [2]: [d_date_sk#128, d_year#129] +Arguments: [c_first_sales_date_sk#127], [d_date_sk#128], Inner, BuildRight -(141) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#143, d_year#144] +(132) CometProject +Input [18]: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, c_first_sales_date_sk#127, d_date_sk#128, d_year#129] +Arguments: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, d_year#129], [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, d_year#129] -(142) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [c_first_shipto_date_sk#139] -Right keys [1]: [d_date_sk#143] -Join type: Inner -Join condition: None +(133) ReusedExchange [Reuses operator id: 48] +Output [2]: [d_date_sk#130, d_year#131] -(143) Project [codegen id : 49] -Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144] -Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, d_year#142, d_date_sk#143, d_year#144] +(134) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, d_year#129] +Right output [2]: [d_date_sk#130, d_year#131] +Arguments: [c_first_shipto_date_sk#126], [d_date_sk#130], Inner, BuildRight -(144) ReusedExchange [Reuses operator id: 61] -Output [2]: [cd_demo_sk#145, cd_marital_status#146] +(135) CometProject +Input [18]: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, d_year#129, d_date_sk#130, d_year#131] +Arguments: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131], [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131] -(145) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [ss_cdemo_sk#108] -Right keys [1]: [cd_demo_sk#145] -Join type: Inner -Join condition: None +(136) ReusedExchange [Reuses operator id: 56] +Output [2]: [cd_demo_sk#132, cd_marital_status#133] -(146) Project [codegen id : 49] -Output [16]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, cd_marital_status#146] -Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, cd_demo_sk#145, cd_marital_status#146] +(137) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131] +Right output [2]: [cd_demo_sk#132, cd_marital_status#133] +Arguments: [ss_cdemo_sk#95], [cd_demo_sk#132], Inner, BuildRight -(147) ReusedExchange [Reuses operator id: 61] -Output [2]: [cd_demo_sk#147, cd_marital_status#148] +(138) CometProject +Input [18]: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, cd_demo_sk#132, cd_marital_status#133] +Arguments: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, cd_marital_status#133], [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, cd_marital_status#133] -(148) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [c_current_cdemo_sk#136] -Right keys [1]: [cd_demo_sk#147] -Join type: Inner -Join condition: NOT (cd_marital_status#146 = cd_marital_status#148) +(139) ReusedExchange [Reuses operator id: 56] +Output [2]: [cd_demo_sk#134, cd_marital_status#135] -(149) Project [codegen id : 49] -Output [14]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144] -Input [18]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, cd_marital_status#146, cd_demo_sk#147, cd_marital_status#148] +(140) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, cd_marital_status#133] +Right output [2]: [cd_demo_sk#134, cd_marital_status#135] +Arguments: [c_current_cdemo_sk#123], [cd_demo_sk#134], Inner, NOT (cd_marital_status#133 = cd_marital_status#135), BuildRight -(150) ReusedExchange [Reuses operator id: 70] -Output [1]: [p_promo_sk#149] +(141) CometProject +Input [18]: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, cd_marital_status#133, cd_demo_sk#134, cd_marital_status#135] +Arguments: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131], [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131] -(151) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [ss_promo_sk#112] -Right keys [1]: [p_promo_sk#149] -Join type: Inner -Join condition: None +(142) ReusedExchange [Reuses operator id: 64] +Output [1]: [p_promo_sk#136] -(152) Project [codegen id : 49] -Output [13]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144] -Input [15]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, p_promo_sk#149] +(143) CometBroadcastHashJoin +Left output [14]: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131] +Right output [1]: [p_promo_sk#136] +Arguments: [ss_promo_sk#99], [p_promo_sk#136], Inner, BuildRight -(153) ReusedExchange [Reuses operator id: 76] -Output [2]: [hd_demo_sk#150, hd_income_band_sk#151] +(144) CometProject +Input [15]: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, p_promo_sk#136] +Arguments: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131], [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131] -(154) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [ss_hdemo_sk#109] -Right keys [1]: [hd_demo_sk#150] -Join type: Inner -Join condition: None +(145) ReusedExchange [Reuses operator id: 69] +Output [2]: [hd_demo_sk#137, hd_income_band_sk#138] -(155) Project [codegen id : 49] -Output [13]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151] -Input [15]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, hd_demo_sk#150, hd_income_band_sk#151] +(146) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131] +Right output [2]: [hd_demo_sk#137, hd_income_band_sk#138] +Arguments: [ss_hdemo_sk#96], [hd_demo_sk#137], Inner, BuildRight -(156) ReusedExchange [Reuses operator id: 76] -Output [2]: [hd_demo_sk#152, hd_income_band_sk#153] +(147) CometProject +Input [15]: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, hd_demo_sk#137, hd_income_band_sk#138] +Arguments: [ss_item_sk#93, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138], [ss_item_sk#93, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138] -(157) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [c_current_hdemo_sk#137] -Right keys [1]: [hd_demo_sk#152] -Join type: Inner -Join condition: None +(148) ReusedExchange [Reuses operator id: 69] +Output [2]: [hd_demo_sk#139, hd_income_band_sk#140] -(158) Project [codegen id : 49] -Output [13]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153] -Input [15]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_demo_sk#152, hd_income_band_sk#153] +(149) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#93, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138] +Right output [2]: [hd_demo_sk#139, hd_income_band_sk#140] +Arguments: [c_current_hdemo_sk#124], [hd_demo_sk#139], Inner, BuildRight -(159) ReusedExchange [Reuses operator id: 85] -Output [5]: [ca_address_sk#154, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] +(150) CometProject +Input [15]: [ss_item_sk#93, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_demo_sk#139, hd_income_band_sk#140] +Arguments: [ss_item_sk#93, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140], [ss_item_sk#93, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140] -(160) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [ss_addr_sk#110] -Right keys [1]: [ca_address_sk#154] -Join type: Inner -Join condition: None +(151) ReusedExchange [Reuses operator id: 77] +Output [5]: [ca_address_sk#141, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] -(161) Project [codegen id : 49] -Output [16]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] -Input [18]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_address_sk#154, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] +(152) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#93, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140] +Right output [5]: [ca_address_sk#141, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Arguments: [ss_addr_sk#97], [ca_address_sk#141], Inner, BuildRight -(162) ReusedExchange [Reuses operator id: 85] -Output [5]: [ca_address_sk#159, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] +(153) CometProject +Input [18]: [ss_item_sk#93, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_address_sk#141, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Arguments: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145], [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] -(163) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [c_current_addr_sk#138] -Right keys [1]: [ca_address_sk#159] -Join type: Inner -Join condition: None +(154) ReusedExchange [Reuses operator id: 77] +Output [5]: [ca_address_sk#146, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] -(164) Project [codegen id : 49] -Output [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] -Input [21]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_address_sk#159, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] +(155) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Right output [5]: [ca_address_sk#146, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] +Arguments: [c_current_addr_sk#125], [ca_address_sk#146], Inner, BuildRight -(165) ReusedExchange [Reuses operator id: 94] -Output [1]: [ib_income_band_sk#164] +(156) CometProject +Input [21]: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_address_sk#146, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] +Arguments: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150], [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] -(166) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [hd_income_band_sk#151] -Right keys [1]: [ib_income_band_sk#164] -Join type: Inner -Join condition: None +(157) ReusedExchange [Reuses operator id: 85] +Output [1]: [ib_income_band_sk#151] -(167) Project [codegen id : 49] -Output [18]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] -Input [20]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, ib_income_band_sk#164] +(158) CometBroadcastHashJoin +Left output [19]: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] +Right output [1]: [ib_income_band_sk#151] +Arguments: [hd_income_band_sk#138], [ib_income_band_sk#151], Inner, BuildRight -(168) ReusedExchange [Reuses operator id: 94] -Output [1]: [ib_income_band_sk#165] +(159) CometProject +Input [20]: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, ib_income_band_sk#151] +Arguments: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150], [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] -(169) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [hd_income_band_sk#153] -Right keys [1]: [ib_income_band_sk#165] -Join type: Inner -Join condition: None +(160) ReusedExchange [Reuses operator id: 85] +Output [1]: [ib_income_band_sk#152] -(170) Project [codegen id : 49] -Output [17]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] -Input [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, ib_income_band_sk#165] +(161) CometBroadcastHashJoin +Left output [18]: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] +Right output [1]: [ib_income_band_sk#152] +Arguments: [hd_income_band_sk#140], [ib_income_band_sk#152], Inner, BuildRight -(171) ReusedExchange [Reuses operator id: 104] -Output [2]: [i_item_sk#166, i_product_name#167] +(162) CometProject +Input [19]: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, ib_income_band_sk#152] +Arguments: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150], [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] -(172) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [ss_item_sk#106] -Right keys [1]: [i_item_sk#166] -Join type: Inner -Join condition: None - -(173) Project [codegen id : 49] -Output [18]: [ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, d_year#142, d_year#144, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, i_item_sk#166, i_product_name#167] -Input [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, i_item_sk#166, i_product_name#167] - -(174) HashAggregate [codegen id : 49] -Input [18]: [ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, d_year#142, d_year#144, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, i_item_sk#166, i_product_name#167] -Keys [15]: [i_product_name#167, i_item_sk#166, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, d_year#131, d_year#142, d_year#144] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#114)), partial_sum(UnscaledValue(ss_list_price#115)), partial_sum(UnscaledValue(ss_coupon_amt#116))] -Aggregate Attributes [4]: [count#77, sum#168, sum#169, sum#170] -Results [19]: [i_product_name#167, i_item_sk#166, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, d_year#131, d_year#142, d_year#144, count#81, sum#171, sum#172, sum#173] - -(175) HashAggregate [codegen id : 49] -Input [19]: [i_product_name#167, i_item_sk#166, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, d_year#131, d_year#142, d_year#144, count#81, sum#171, sum#172, sum#173] -Keys [15]: [i_product_name#167, i_item_sk#166, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, d_year#131, d_year#142, d_year#144] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#114)), sum(UnscaledValue(ss_list_price#115)), sum(UnscaledValue(ss_coupon_amt#116))] -Aggregate Attributes [4]: [count(1)#85, sum(UnscaledValue(ss_wholesale_cost#114))#86, sum(UnscaledValue(ss_list_price#115))#87, sum(UnscaledValue(ss_coupon_amt#116))#88] -Results [8]: [i_item_sk#166 AS item_sk#174, s_store_name#133 AS store_name#175, s_zip#134 AS store_zip#176, d_year#131 AS syear#177, count(1)#85 AS cnt#178, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#114))#86,17,2) AS s1#179, MakeDecimal(sum(UnscaledValue(ss_list_price#115))#87,17,2) AS s2#180, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#116))#88,17,2) AS s3#181] - -(176) Exchange -Input [8]: [item_sk#174, store_name#175, store_zip#176, syear#177, cnt#178, s1#179, s2#180, s3#181] -Arguments: hashpartitioning(item_sk#174, store_name#175, store_zip#176, 5), ENSURE_REQUIREMENTS, [plan_id=16] - -(177) Sort [codegen id : 50] -Input [8]: [item_sk#174, store_name#175, store_zip#176, syear#177, cnt#178, s1#179, s2#180, s3#181] -Arguments: [item_sk#174 ASC NULLS FIRST, store_name#175 ASC NULLS FIRST, store_zip#176 ASC NULLS FIRST], false, 0 - -(178) SortMergeJoin [codegen id : 51] -Left keys [3]: [item_sk#90, store_name#91, store_zip#92] -Right keys [3]: [item_sk#174, store_name#175, store_zip#176] +(163) ReusedExchange [Reuses operator id: 94] +Output [2]: [i_item_sk#153, i_product_name#154] + +(164) CometBroadcastHashJoin +Left output [17]: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] +Right output [2]: [i_item_sk#153, i_product_name#154] +Arguments: [ss_item_sk#93], [i_item_sk#153], Inner, BuildRight + +(165) CometProject +Input [19]: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, i_item_sk#153, i_product_name#154] +Arguments: [ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, d_year#129, d_year#131, s_store_name#120, s_zip#121, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, i_item_sk#153, i_product_name#154], [ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, d_year#129, d_year#131, s_store_name#120, s_zip#121, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, i_item_sk#153, i_product_name#154] + +(166) CometHashAggregate +Input [18]: [ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, d_year#129, d_year#131, s_store_name#120, s_zip#121, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, i_item_sk#153, i_product_name#154] +Keys [15]: [i_product_name#154, i_item_sk#153, s_store_name#120, s_zip#121, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, d_year#118, d_year#129, d_year#131] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#101)), partial_sum(UnscaledValue(ss_list_price#102)), partial_sum(UnscaledValue(ss_coupon_amt#103))] + +(167) CometHashAggregate +Input [19]: [i_product_name#154, i_item_sk#153, s_store_name#120, s_zip#121, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, d_year#118, d_year#129, d_year#131, count#72, sum#155, sum#156, sum#157] +Keys [15]: [i_product_name#154, i_item_sk#153, s_store_name#120, s_zip#121, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, d_year#118, d_year#129, d_year#131] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#101)), sum(UnscaledValue(ss_list_price#102)), sum(UnscaledValue(ss_coupon_amt#103))] + +(168) CometColumnarExchange +Input [8]: [item_sk#158, store_name#159, store_zip#160, syear#161, cnt#162, s1#163, s2#164, s3#165] +Arguments: hashpartitioning(item_sk#158, store_name#159, store_zip#160, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(169) CometSort +Input [8]: [item_sk#158, store_name#159, store_zip#160, syear#161, cnt#162, s1#163, s2#164, s3#165] +Arguments: [item_sk#158, store_name#159, store_zip#160, syear#161, cnt#162, s1#163, s2#164, s3#165], [item_sk#158 ASC NULLS FIRST, store_name#159 ASC NULLS FIRST, store_zip#160 ASC NULLS FIRST] + +(170) ColumnarToRow [codegen id : 2] +Input [8]: [item_sk#158, store_name#159, store_zip#160, syear#161, cnt#162, s1#163, s2#164, s3#165] + +(171) SortMergeJoin [codegen id : 3] +Left keys [3]: [item_sk#77, store_name#78, store_zip#79] +Right keys [3]: [item_sk#158, store_name#159, store_zip#160] Join type: Inner -Join condition: (cnt#178 <= cnt#102) +Join condition: (cnt#162 <= cnt#89) -(179) Project [codegen id : 51] -Output [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#179, s2#180, s3#181, syear#177, cnt#178] -Input [25]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, item_sk#174, store_name#175, store_zip#176, syear#177, cnt#178, s1#179, s2#180, s3#181] +(172) Project [codegen id : 3] +Output [21]: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162] +Input [25]: [product_name#76, item_sk#77, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, item_sk#158, store_name#159, store_zip#160, syear#161, cnt#162, s1#163, s2#164, s3#165] -(180) Exchange -Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#179, s2#180, s3#181, syear#177, cnt#178] -Arguments: rangepartitioning(product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#178 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=17] +(173) RowToColumnar +Input [21]: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162] -(181) Sort [codegen id : 52] -Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#179, s2#180, s3#181, syear#177, cnt#178] -Arguments: [product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#178 ASC NULLS FIRST], true, 0 +(174) CometColumnarExchange +Input [21]: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162] +Arguments: rangepartitioning(product_name#76 ASC NULLS FIRST, store_name#78 ASC NULLS FIRST, cnt#162 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(175) CometSort +Input [21]: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162] +Arguments: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162], [product_name#76 ASC NULLS FIRST, store_name#78 ASC NULLS FIRST, cnt#162 ASC NULLS FIRST] + +(176) ColumnarToRow [codegen id : 4] +Input [21]: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (185) -+- * ColumnarToRow (184) - +- CometFilter (183) - +- CometScan parquet spark_catalog.default.date_dim (182) +BroadcastExchange (180) ++- * ColumnarToRow (179) + +- CometFilter (178) + +- CometScan parquet spark_catalog.default.date_dim (177) -(182) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#37, d_year#38] +(177) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#32, d_year#33] 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 -(183) CometFilter -Input [2]: [d_date_sk#37, d_year#38] -Condition : ((isnotnull(d_year#38) AND (d_year#38 = 1999)) AND isnotnull(d_date_sk#37)) +(178) CometFilter +Input [2]: [d_date_sk#32, d_year#33] +Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(184) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#37, d_year#38] +(179) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#32, d_year#33] -(185) BroadcastExchange -Input [2]: [d_date_sk#37, d_year#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] +(180) BroadcastExchange +Input [2]: [d_date_sk#32, d_year#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -Subquery:2 Hosting operator id = 111 Hosting Expression = ss_sold_date_sk#117 IN dynamicpruning#118 -BroadcastExchange (189) -+- * ColumnarToRow (188) - +- CometFilter (187) - +- CometScan parquet spark_catalog.default.date_dim (186) +Subquery:2 Hosting operator id = 102 Hosting Expression = ss_sold_date_sk#104 IN dynamicpruning#105 +BroadcastExchange (184) ++- * ColumnarToRow (183) + +- CometFilter (182) + +- CometScan parquet spark_catalog.default.date_dim (181) -(186) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#130, d_year#131] +(181) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#117, d_year#118] 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 -(187) CometFilter -Input [2]: [d_date_sk#130, d_year#131] -Condition : ((isnotnull(d_year#131) AND (d_year#131 = 2000)) AND isnotnull(d_date_sk#130)) +(182) CometFilter +Input [2]: [d_date_sk#117, d_year#118] +Condition : ((isnotnull(d_year#118) AND (d_year#118 = 2000)) AND isnotnull(d_date_sk#117)) -(188) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#130, d_year#131] +(183) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#117, d_year#118] -(189) BroadcastExchange -Input [2]: [d_date_sk#130, d_year#131] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=19] +(184) BroadcastExchange +Input [2]: [d_date_sk#117, d_year#118] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt index 3b5e4f19e..84be0e3c3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt @@ -1,281 +1,199 @@ -WholeStageCodegen (52) - Sort [product_name,store_name,cnt] +WholeStageCodegen (4) + ColumnarToRow InputAdapter - Exchange [product_name,store_name,cnt] #1 - WholeStageCodegen (51) - Project [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - SortMergeJoin [item_sk,store_name,store_zip,item_sk,store_name,store_zip,cnt,cnt] - InputAdapter - WholeStageCodegen (25) - Sort [item_sk,store_name,store_zip] - InputAdapter - Exchange [item_sk,store_name,store_zip] #2 - WholeStageCodegen (24) - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] - Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SortMergeJoin [ss_item_sk,cs_item_sk] - InputAdapter - WholeStageCodegen (2) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #4 - CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - WholeStageCodegen (8) - Sort [cs_item_sk] - Project [cs_item_sk] - Filter [sale,refund] - HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] - InputAdapter - Exchange [cs_item_sk] #6 - WholeStageCodegen (7) - HashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [sum,sum,isEmpty,sum,sum,isEmpty] - Project [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] - InputAdapter - WholeStageCodegen (4) - Sort [cs_item_sk,cs_order_number] - InputAdapter - Exchange [cs_item_sk,cs_order_number] #7 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - InputAdapter - WholeStageCodegen (6) - Sort [cr_item_sk,cr_order_number] - InputAdapter - Exchange [cr_item_sk,cr_order_number] #8 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_name,s_zip] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (12) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #11 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (14) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk,cd_marital_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometFilter [p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk] - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (17) - 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 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (19) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 - InputAdapter - BroadcastExchange #16 - WholeStageCodegen (21) - ColumnarToRow - InputAdapter - CometFilter [ib_income_band_sk] - CometScan parquet spark_catalog.default.income_band [ib_income_band_sk] - InputAdapter - ReusedExchange [ib_income_band_sk] #16 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (23) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] - InputAdapter - WholeStageCodegen (50) - Sort [item_sk,store_name,store_zip] - InputAdapter - Exchange [item_sk,store_name,store_zip] #18 - WholeStageCodegen (49) - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] - Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SortMergeJoin [ss_item_sk,cs_item_sk] - InputAdapter - WholeStageCodegen (27) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #19 - WholeStageCodegen (26) - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #20 - CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #21 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - WholeStageCodegen (33) - Sort [cs_item_sk] - Project [cs_item_sk] - Filter [sale,refund] - HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] - InputAdapter - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #21 - InputAdapter - ReusedExchange [s_store_sk,s_store_name,s_zip] #9 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #10 - InputAdapter - ReusedExchange [d_date_sk,d_year] #11 - InputAdapter - ReusedExchange [d_date_sk,d_year] #11 - InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 - InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 - InputAdapter - ReusedExchange [p_promo_sk] #13 - InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 - InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 - InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 - InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 - InputAdapter - ReusedExchange [ib_income_band_sk] #16 - InputAdapter - ReusedExchange [ib_income_band_sk] #16 - InputAdapter - ReusedExchange [i_item_sk,i_product_name] #17 + CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] + CometColumnarExchange [product_name,store_name,cnt] #1 + RowToColumnar + WholeStageCodegen (3) + Project [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] + SortMergeJoin [item_sk,store_name,store_zip,item_sk,store_name,store_zip,cnt,cnt] + InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometSort [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3] + CometColumnarExchange [item_sk,store_name,store_zip] #2 + CometHashAggregate [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] + CometHashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum,ss_wholesale_cost,ss_list_price,ss_coupon_amt] + CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] + CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk] #3 + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #4 + CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [cs_item_sk,sale,refund] + CometHashAggregate [cs_item_sk,sale,refund,sum,sum,isEmpty,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] + CometColumnarExchange [cs_item_sk] #6 + CometHashAggregate [cs_item_sk,sum,sum,isEmpty,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometSortMergeJoin [cs_item_sk,cs_order_number,cs_ext_list_price,cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometSort [cs_item_sk,cs_order_number,cs_ext_list_price] + CometColumnarExchange [cs_item_sk,cs_order_number] #7 + CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometColumnarExchange [cr_item_sk,cr_order_number] #8 + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 + CometFilter [s_store_sk,s_store_name,s_zip] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #12 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk,d_year] #12 + CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 + CometFilter [cd_demo_sk,cd_marital_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + ReusedExchange [cd_demo_sk,cd_marital_status] #13 + CometBroadcastExchange [p_promo_sk] #14 + CometFilter [p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk] + CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 + CometFilter [hd_demo_sk,hd_income_band_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] + ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 + CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + CometBroadcastExchange [ib_income_band_sk] #17 + CometFilter [ib_income_band_sk] + CometScan parquet spark_catalog.default.income_band [ib_income_band_sk] + ReusedExchange [ib_income_band_sk] #17 + CometBroadcastExchange [i_item_sk,i_product_name] #18 + CometProject [i_item_sk,i_product_name] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] + InputAdapter + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometSort [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] + CometColumnarExchange [item_sk,store_name,store_zip] #19 + CometHashAggregate [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] + CometHashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum,ss_wholesale_cost,ss_list_price,ss_coupon_amt] + CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] + CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk] #20 + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #21 + CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #22 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [cs_item_sk,sale,refund] + CometHashAggregate [cs_item_sk,sale,refund,sum,sum,isEmpty,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] + ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 + CometBroadcastExchange [d_date_sk,d_year] #23 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [s_store_sk,s_store_name,s_zip] #10 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 + ReusedExchange [d_date_sk,d_year] #12 + ReusedExchange [d_date_sk,d_year] #12 + ReusedExchange [cd_demo_sk,cd_marital_status] #13 + ReusedExchange [cd_demo_sk,cd_marital_status] #13 + ReusedExchange [p_promo_sk] #14 + ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 + ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + ReusedExchange [ib_income_band_sk] #17 + ReusedExchange [ib_income_band_sk] #17 + ReusedExchange [i_item_sk,i_product_name] #18 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 a10b6897c..dd1796dd4 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,46 +1,43 @@ == Physical Plan == -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 (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) +* ColumnarToRow (39) ++- CometTakeOrderedAndProject (38) + +- CometProject (37) + +- CometBroadcastHashJoin (36) + :- CometProject (22) + : +- CometBroadcastHashJoin (21) + : :- CometProject (17) + : : +- CometBroadcastHashJoin (16) + : : :- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store (1) + : : +- CometBroadcastExchange (15) + : : +- CometFilter (14) + : : +- CometHashAggregate (13) + : : +- CometColumnarExchange (12) + : : +- CometHashAggregate (11) + : : +- 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 (20) + : +- CometFilter (19) + : +- CometScan parquet spark_catalog.default.item (18) + +- CometBroadcastExchange (35) + +- CometFilter (34) + +- CometHashAggregate (33) + +- CometColumnarExchange (32) + +- CometHashAggregate (31) + +- CometHashAggregate (30) + +- CometColumnarExchange (29) + +- CometHashAggregate (28) + +- CometProject (27) + +- CometBroadcastHashJoin (26) + :- CometFilter (24) + : +- CometScan parquet spark_catalog.default.store_sales (23) + +- ReusedExchange (25) (1) Scan parquet spark_catalog.default.store @@ -54,10 +51,7 @@ ReadSchema: struct Input [2]: [s_store_sk#1, s_store_name#2] Condition : isnotnull(s_store_sk#1) -(3) ColumnarToRow [codegen id : 7] -Input [2]: [s_store_sk#1, s_store_name#2] - -(4) Scan parquet spark_catalog.default.store_sales +(3) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] Batched: true Location: InMemoryFileIndex [] @@ -65,218 +59,201 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter 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) Scan parquet spark_catalog.default.date_dim +(5) 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) CometFilter +(6) 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) CometProject +(7) CometProject Input [2]: [d_date_sk#8, d_month_seq#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(9) CometBroadcastExchange +(8) CometBroadcastExchange Input [1]: [d_date_sk#8] Arguments: [d_date_sk#8] -(10) CometBroadcastHashJoin +(9) 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 +(10) 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) CometHashAggregate +(11) 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))] -(13) ColumnarToRow [codegen id : 1] -Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] - -(14) Exchange +(12) CometColumnarExchange 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] +Arguments: hashpartitioning(ss_store_sk#4, ss_item_sk#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(15) HashAggregate [codegen id : 2] +(13) CometHashAggregate 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] -(16) Filter [codegen id : 2] -Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#12] -Condition : isnotnull(revenue#12) +(14) CometFilter +Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] +Condition : isnotnull(revenue#11) -(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) CometBroadcastExchange +Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] +Arguments: [ss_store_sk#4, ss_item_sk#3, revenue#11] -(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) CometBroadcastHashJoin +Left output [2]: [s_store_sk#1, s_store_name#2] +Right output [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] +Arguments: [s_store_sk#1], [ss_store_sk#4], Inner, BuildRight -(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) CometProject +Input [5]: [s_store_sk#1, s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11] +Arguments: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11], [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11] -(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] +(18) Scan parquet spark_catalog.default.item +Output [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(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) CometFilter +Input [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] +Condition : isnotnull(i_item_sk#12) -(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) CometBroadcastExchange +Input [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] +Arguments: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] -(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) CometBroadcastHashJoin +Left output [4]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11] +Right output [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] +Arguments: [ss_item_sk#3], [i_item_sk#12], Inner, BuildRight -(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) CometProject +Input [9]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11, i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] +Arguments: [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16], [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] -(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] - -(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] +(23) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_item_sk#17, ss_store_sk#18, ss_sales_price#19, ss_sold_date_sk#20] 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#20), dynamicpruningexpression(ss_sold_date_sk#20 IN dynamicpruning#21)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(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) +(24) CometFilter +Input [4]: [ss_item_sk#17, ss_store_sk#18, ss_sales_price#19, ss_sold_date_sk#20] +Condition : isnotnull(ss_store_sk#18) + +(25) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#22] + +(26) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#17, ss_store_sk#18, ss_sales_price#19, ss_sold_date_sk#20] +Right output [1]: [d_date_sk#22] +Arguments: [ss_sold_date_sk#20], [d_date_sk#22], Inner, BuildRight + +(27) CometProject +Input [5]: [ss_item_sk#17, ss_store_sk#18, ss_sales_price#19, ss_sold_date_sk#20, d_date_sk#22] +Arguments: [ss_item_sk#17, ss_store_sk#18, ss_sales_price#19], [ss_item_sk#17, ss_store_sk#18, ss_sales_price#19] -(28) ReusedExchange [Reuses operator id: 9] -Output [1]: [d_date_sk#23] +(28) CometHashAggregate +Input [3]: [ss_item_sk#17, ss_store_sk#18, ss_sales_price#19] +Keys [2]: [ss_store_sk#18, ss_item_sk#17] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#19))] -(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 +(29) CometColumnarExchange +Input [3]: [ss_store_sk#18, ss_item_sk#17, sum#23] +Arguments: hashpartitioning(ss_store_sk#18, ss_item_sk#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(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] +(30) CometHashAggregate +Input [3]: [ss_store_sk#18, ss_item_sk#17, sum#23] +Keys [2]: [ss_store_sk#18, ss_item_sk#17] +Functions [1]: [sum(UnscaledValue(ss_sales_price#19))] (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))] - -(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] - -(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))#25] -Results [2]: [ss_store_sk#19, MakeDecimal(sum(UnscaledValue(ss_sales_price#20))#25,17,2) AS revenue#26] - -(35) HashAggregate [codegen id : 5] -Input [2]: [ss_store_sk#19, revenue#26] -Keys [1]: [ss_store_sk#19] -Functions [1]: [partial_avg(revenue#26)] -Aggregate Attributes [2]: [sum#27, count#28] -Results [3]: [ss_store_sk#19, sum#29, count#30] - -(36) Exchange -Input [3]: [ss_store_sk#19, sum#29, count#30] -Arguments: hashpartitioning(ss_store_sk#19, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(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#26)] -Aggregate Attributes [1]: [avg(revenue#26)#31] -Results [2]: [ss_store_sk#19, avg(revenue#26)#31 AS ave#32] - -(38) Filter [codegen id : 6] -Input [2]: [ss_store_sk#19, ave#32] -Condition : isnotnull(ave#32) - -(39) BroadcastExchange -Input [2]: [ss_store_sk#19, ave#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -(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#32)) - -(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#32] - -(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] +Input [2]: [ss_store_sk#18, revenue#24] +Keys [1]: [ss_store_sk#18] +Functions [1]: [partial_avg(revenue#24)] + +(32) CometColumnarExchange +Input [3]: [ss_store_sk#18, sum#25, count#26] +Arguments: hashpartitioning(ss_store_sk#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(33) CometHashAggregate +Input [3]: [ss_store_sk#18, sum#25, count#26] +Keys [1]: [ss_store_sk#18] +Functions [1]: [avg(revenue#24)] + +(34) CometFilter +Input [2]: [ss_store_sk#18, ave#27] +Condition : isnotnull(ave#27) + +(35) CometBroadcastExchange +Input [2]: [ss_store_sk#18, ave#27] +Arguments: [ss_store_sk#18, ave#27] + +(36) CometBroadcastHashJoin +Left output [7]: [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] +Right output [2]: [ss_store_sk#18, ave#27] +Arguments: [ss_store_sk#4], [ss_store_sk#18], Inner, (cast(revenue#11 as decimal(23,7)) <= (0.1 * ave#27)), BuildRight + +(37) CometProject +Input [9]: [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16, ss_store_sk#18, ave#27] +Arguments: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#16], [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#16] + +(38) CometTakeOrderedAndProject +Input [6]: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#16] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#2 ASC NULLS FIRST,i_item_desc#13 ASC NULLS FIRST], output=[s_store_name#2,i_item_desc#13,revenue#11,i_current_price#14,i_wholesale_cost#15,i_brand#16]), [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#16], 100, [s_store_name#2 ASC NULLS FIRST, i_item_desc#13 ASC NULLS FIRST], [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#16] + +(39) ColumnarToRow [codegen id : 1] +Input [6]: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#16] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -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#6 IN dynamicpruning#7 +BroadcastExchange (44) ++- * ColumnarToRow (43) + +- CometProject (42) + +- CometFilter (41) + +- CometScan parquet spark_catalog.default.date_dim (40) -(43) Scan parquet spark_catalog.default.date_dim +(40) 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 -(44) CometFilter +(41) 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)) -(45) CometProject +(42) CometProject Input [2]: [d_date_sk#8, d_month_seq#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(46) ColumnarToRow [codegen id : 1] +(43) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#8] -(47) BroadcastExchange +(44) BroadcastExchange Input [1]: [d_date_sk#8] -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=4] -Subquery:2 Hosting operator id = 26 Hosting Expression = ss_sold_date_sk#21 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#20 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 6cb247973..e41f9161d 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,68 +1,50 @@ -TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - 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] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [s_store_name,ss_store_sk,ss_item_sk,revenue] - BroadcastHashJoin [s_store_sk,ss_store_sk] - ColumnarToRow - InputAdapter +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] + CometProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] + CometBroadcastHashJoin [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand,ss_store_sk,ave] + CometProject [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand] + CometBroadcastHashJoin [s_store_name,ss_store_sk,ss_item_sk,revenue,i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] + CometProject [s_store_name,ss_store_sk,ss_item_sk,revenue] + CometBroadcastHashJoin [s_store_sk,s_store_name,ss_store_sk,ss_item_sk,revenue] CometFilter [s_store_sk,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] - InputAdapter - BroadcastExchange #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ss_store_sk,ss_item_sk,sum,ss_sales_price] - CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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 #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - Filter [ave] - HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count] - InputAdapter - 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] #8 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometHashAggregate [ss_store_sk,ss_item_sk,sum,ss_sales_price] - CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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 - ReusedExchange [d_date_sk] #4 + CometBroadcastExchange [ss_store_sk,ss_item_sk,revenue] #1 + CometFilter [ss_store_sk,ss_item_sk,revenue] + CometHashAggregate [ss_store_sk,ss_item_sk,revenue,sum,sum(UnscaledValue(ss_sales_price))] + CometColumnarExchange [ss_store_sk,ss_item_sk] #2 + CometHashAggregate [ss_store_sk,ss_item_sk,sum,ss_sales_price] + CometProject [ss_item_sk,ss_store_sk,ss_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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 #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] #5 + CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] + CometBroadcastExchange [ss_store_sk,ave] #6 + CometFilter [ss_store_sk,ave] + CometHashAggregate [ss_store_sk,ave,sum,count,avg(revenue)] + CometColumnarExchange [ss_store_sk] #7 + CometHashAggregate [ss_store_sk,sum,count,revenue] + CometHashAggregate [ss_store_sk,revenue,ss_item_sk,sum,sum(UnscaledValue(ss_sales_price))] + CometColumnarExchange [ss_store_sk,ss_item_sk] #8 + CometHashAggregate [ss_store_sk,ss_item_sk,sum,ss_sales_price] + CometProject [ss_item_sk,ss_store_sk,ss_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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 + 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 6bca75e27..d0c1814d1 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,55 +1,54 @@ == Physical Plan == -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) +* ColumnarToRow (50) ++- CometTakeOrderedAndProject (49) + +- CometHashAggregate (48) + +- CometColumnarExchange (47) + +- CometHashAggregate (46) + +- CometUnion (45) + :- CometHashAggregate (27) + : +- CometColumnarExchange (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) + +- CometHashAggregate (44) + +- CometColumnarExchange (43) + +- CometHashAggregate (42) + +- CometProject (41) + +- CometBroadcastHashJoin (40) + :- CometProject (38) + : +- CometBroadcastHashJoin (37) + : :- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometProject (32) + : : : +- CometBroadcastHashJoin (31) + : : : :- CometFilter (29) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (28) + : : : +- ReusedExchange (30) + : : +- ReusedExchange (33) + : +- ReusedExchange (36) + +- ReusedExchange (39) (1) Scan parquet spark_catalog.default.web_sales @@ -173,150 +172,141 @@ Input [11]: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_nam 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)] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(28) HashAggregate [codegen id : 2] +(27) CometHashAggregate 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)#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] -(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] +(28) Scan parquet spark_catalog.default.catalog_sales +Output [7]: [cs_sold_time_sk#71, cs_ship_mode_sk#72, cs_warehouse_sk#73, cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, cs_sold_date_sk#77] 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#77), dynamicpruningexpression(cs_sold_date_sk#77 IN dynamicpruning#78)] PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_sold_time_sk), IsNotNull(cs_ship_mode_sk)] ReadSchema: struct -(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] +(29) CometFilter +Input [7]: [cs_sold_time_sk#71, cs_ship_mode_sk#72, cs_warehouse_sk#73, cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, cs_sold_date_sk#77] +Condition : ((isnotnull(cs_warehouse_sk#73) AND isnotnull(cs_sold_time_sk#71)) AND isnotnull(cs_ship_mode_sk#72)) + +(30) ReusedExchange [Reuses operator id: 5] +Output [7]: [w_warehouse_sk#79, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85] + +(31) CometBroadcastHashJoin +Left output [7]: [cs_sold_time_sk#71, cs_ship_mode_sk#72, cs_warehouse_sk#73, cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, cs_sold_date_sk#77] +Right output [7]: [w_warehouse_sk#79, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85] +Arguments: [cs_warehouse_sk#73], [w_warehouse_sk#79], Inner, BuildRight + +(32) CometProject +Input [14]: [cs_sold_time_sk#71, cs_ship_mode_sk#72, cs_warehouse_sk#73, cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, cs_sold_date_sk#77, w_warehouse_sk#79, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85] +Arguments: [cs_sold_time_sk#71, cs_ship_mode_sk#72, cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, cs_sold_date_sk#77, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85], [cs_sold_time_sk#71, cs_ship_mode_sk#72, cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, cs_sold_date_sk#77, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85] + +(33) ReusedExchange [Reuses operator id: 10] +Output [3]: [d_date_sk#86, d_year#87, d_moy#88] + +(34) CometBroadcastHashJoin +Left output [12]: [cs_sold_time_sk#71, cs_ship_mode_sk#72, cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, cs_sold_date_sk#77, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85] +Right output [3]: [d_date_sk#86, d_year#87, d_moy#88] +Arguments: [cs_sold_date_sk#77], [d_date_sk#86], Inner, BuildRight + +(35) CometProject +Input [15]: [cs_sold_time_sk#71, cs_ship_mode_sk#72, cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, cs_sold_date_sk#77, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, d_date_sk#86, d_year#87, d_moy#88] +Arguments: [cs_sold_time_sk#71, cs_ship_mode_sk#72, cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, d_year#87, d_moy#88], [cs_sold_time_sk#71, cs_ship_mode_sk#72, cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, d_year#87, d_moy#88] + +(36) ReusedExchange [Reuses operator id: 16] +Output [1]: [t_time_sk#89] + +(37) CometBroadcastHashJoin +Left output [13]: [cs_sold_time_sk#71, cs_ship_mode_sk#72, cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, d_year#87, d_moy#88] +Right output [1]: [t_time_sk#89] +Arguments: [cs_sold_time_sk#71], [t_time_sk#89], Inner, BuildRight + +(38) CometProject +Input [14]: [cs_sold_time_sk#71, cs_ship_mode_sk#72, cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, d_year#87, d_moy#88, t_time_sk#89] +Arguments: [cs_ship_mode_sk#72, cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, d_year#87, d_moy#88], [cs_ship_mode_sk#72, cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, d_year#87, d_moy#88] + +(39) ReusedExchange [Reuses operator id: 22] +Output [1]: [sm_ship_mode_sk#90] + +(40) CometBroadcastHashJoin +Left output [12]: [cs_ship_mode_sk#72, cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, d_year#87, d_moy#88] +Right output [1]: [sm_ship_mode_sk#90] +Arguments: [cs_ship_mode_sk#72], [sm_ship_mode_sk#90], Inner, BuildRight + +(41) CometProject +Input [13]: [cs_ship_mode_sk#72, cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, d_year#87, d_moy#88, sm_ship_mode_sk#90] +Arguments: [cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, d_year#87, d_moy#88], [cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, d_year#87, d_moy#88] + +(42) CometHashAggregate +Input [11]: [cs_quantity#74, cs_sales_price#75, cs_net_paid_inc_tax#76, w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, d_year#87, d_moy#88] +Keys [7]: [w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, d_year#87] +Functions [24]: [partial_sum(CASE WHEN (d_moy#88 = 1) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 2) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 3) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 4) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 5) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 6) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 7) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 8) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 9) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 10) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 11) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 12) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 1) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 2) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 3) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 4) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 5) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 6) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 7) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 8) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 9) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 10) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 11) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#88 = 12) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END)] + +(43) CometColumnarExchange +Input [55]: [w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, d_year#87, 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, sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] +Arguments: hashpartitioning(w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, d_year#87, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(44) CometHashAggregate +Input [55]: [w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, d_year#87, 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, sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] +Keys [7]: [w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, d_year#87] +Functions [24]: [sum(CASE WHEN (d_moy#88 = 1) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 2) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 3) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 4) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 5) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 6) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 7) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 8) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 9) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 10) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 11) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 12) THEN (cs_sales_price#75 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 1) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 2) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 3) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 4) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 5) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 6) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 7) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 8) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 9) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 10) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 11) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#88 = 12) THEN (cs_net_paid_inc_tax#76 * cast(cs_quantity#74 as decimal(10,0))) ELSE 0.00 END)] + +(45) CometUnion +Child 0 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#139, year#140, jan_sales#141, feb_sales#142, mar_sales#143, apr_sales#144, may_sales#145, jun_sales#146, jul_sales#147, aug_sales#148, sep_sales#149, oct_sales#150, nov_sales#151, dec_sales#152, jan_net#153, feb_net#154, mar_net#155, apr_net#156, may_net#157, jun_net#158, jul_net#159, aug_net#160, sep_net#161, oct_net#162, nov_net#163, dec_net#164] +Child 1 Input [32]: [w_warehouse_name#80, w_warehouse_sq_ft#81, w_city#82, w_county#83, w_state#84, w_country#85, ship_carriers#165, year#166, jan_sales#167, feb_sales#168, mar_sales#169, apr_sales#170, may_sales#171, jun_sales#172, jul_sales#173, aug_sales#174, sep_sales#175, oct_sales#176, nov_sales#177, dec_sales#178, jan_net#179, feb_net#180, mar_net#181, apr_net#182, may_net#183, jun_net#184, jul_net#185, aug_net#186, sep_net#187, oct_net#188, nov_net#189, dec_net#190] + +(46) CometHashAggregate +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#139, year#140, jan_sales#141, feb_sales#142, mar_sales#143, apr_sales#144, may_sales#145, jun_sales#146, jul_sales#147, aug_sales#148, sep_sales#149, oct_sales#150, nov_sales#151, dec_sales#152, jan_net#153, feb_net#154, mar_net#155, apr_net#156, may_net#157, jun_net#158, jul_net#159, aug_net#160, sep_net#161, oct_net#162, nov_net#163, dec_net#164] +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#139, year#140] +Functions [36]: [partial_sum(jan_sales#141), partial_sum(feb_sales#142), partial_sum(mar_sales#143), partial_sum(apr_sales#144), partial_sum(may_sales#145), partial_sum(jun_sales#146), partial_sum(jul_sales#147), partial_sum(aug_sales#148), partial_sum(sep_sales#149), partial_sum(oct_sales#150), partial_sum(nov_sales#151), partial_sum(dec_sales#152), partial_sum((jan_sales#141 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((feb_sales#142 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((mar_sales#143 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((apr_sales#144 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((may_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jun_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jul_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((aug_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((sep_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((oct_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((nov_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((dec_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum(jan_net#153), partial_sum(feb_net#154), partial_sum(mar_net#155), partial_sum(apr_net#156), partial_sum(may_net#157), partial_sum(jun_net#158), partial_sum(jul_net#159), partial_sum(aug_net#160), partial_sum(sep_net#161), partial_sum(oct_net#162), partial_sum(nov_net#163), partial_sum(dec_net#164)] + +(47) CometColumnarExchange +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#139, year#140, 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, 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] +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#139, year#140, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(48) CometHashAggregate +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#139, year#140, 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, 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] +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#139, year#140] +Functions [36]: [sum(jan_sales#141), sum(feb_sales#142), sum(mar_sales#143), sum(apr_sales#144), sum(may_sales#145), sum(jun_sales#146), sum(jul_sales#147), sum(aug_sales#148), sum(sep_sales#149), sum(oct_sales#150), sum(nov_sales#151), sum(dec_sales#152), sum((jan_sales#141 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((feb_sales#142 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((mar_sales#143 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((apr_sales#144 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((may_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jun_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jul_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((aug_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((sep_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((oct_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((nov_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((dec_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum(jan_net#153), sum(feb_net#154), sum(mar_net#155), sum(apr_net#156), sum(may_net#157), sum(jun_net#158), sum(jul_net#159), sum(aug_net#160), sum(sep_net#161), sum(oct_net#162), sum(nov_net#163), sum(dec_net#164)] + +(49) CometTakeOrderedAndProject +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#139, year#140, jan_sales#263, feb_sales#264, mar_sales#265, apr_sales#266, may_sales#267, jun_sales#268, jul_sales#269, aug_sales#270, sep_sales#271, oct_sales#272, nov_sales#273, dec_sales#274, jan_sales_per_sq_foot#275, feb_sales_per_sq_foot#276, mar_sales_per_sq_foot#277, apr_sales_per_sq_foot#278, may_sales_per_sq_foot#279, jun_sales_per_sq_foot#280, jul_sales_per_sq_foot#281, aug_sales_per_sq_foot#282, sep_sales_per_sq_foot#283, oct_sales_per_sq_foot#284, nov_sales_per_sq_foot#285, dec_sales_per_sq_foot#286, jan_net#287, feb_net#288, mar_net#289, apr_net#290, may_net#291, jun_net#292, jul_net#293, aug_net#294, sep_net#295, oct_net#296, nov_net#297, dec_net#298] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_warehouse_name#10 ASC NULLS FIRST], output=[w_warehouse_name#10,w_warehouse_sq_ft#11,w_city#12,w_county#13,w_state#14,w_country#15,ship_carriers#139,year#140,jan_sales#263,feb_sales#264,mar_sales#265,apr_sales#266,may_sales#267,jun_sales#268,jul_sales#269,aug_sales#270,sep_sales#271,oct_sales#272,nov_sales#273,dec_sales#274,jan_sales_per_sq_foot#275,feb_sales_per_sq_foot#276,mar_sales_per_sq_foot#277,apr_sales_per_sq_foot#278,may_sales_per_sq_foot#279,jun_sales_per_sq_foot#280,jul_sales_per_sq_foot#281,aug_sales_per_sq_foot#282,sep_sales_per_sq_foot#283,oct_sales_per_sq_foot#284,nov_sales_per_sq_foot#285,dec_sales_per_sq_foot#286,jan_net#287,feb_net#288,mar_net#289,apr_net#290,may_net#291,jun_net#292,jul_net#293,aug_net#294,sep_net#295,oct_net#296,nov_net#297,dec_net#298]), [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#139, year#140, jan_sales#263, feb_sales#264, mar_sales#265, apr_sales#266, may_sales#267, jun_sales#268, jul_sales#269, aug_sales#270, sep_sales#271, oct_sales#272, nov_sales#273, dec_sales#274, jan_sales_per_sq_foot#275, feb_sales_per_sq_foot#276, mar_sales_per_sq_foot#277, apr_sales_per_sq_foot#278, may_sales_per_sq_foot#279, jun_sales_per_sq_foot#280, jul_sales_per_sq_foot#281, aug_sales_per_sq_foot#282, sep_sales_per_sq_foot#283, oct_sales_per_sq_foot#284, nov_sales_per_sq_foot#285, dec_sales_per_sq_foot#286, jan_net#287, feb_net#288, mar_net#289, apr_net#290, may_net#291, jun_net#292, jul_net#293, aug_net#294, sep_net#295, oct_net#296, nov_net#297, dec_net#298], 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#139, year#140, jan_sales#263, feb_sales#264, mar_sales#265, apr_sales#266, may_sales#267, jun_sales#268, jul_sales#269, aug_sales#270, sep_sales#271, oct_sales#272, nov_sales#273, dec_sales#274, jan_sales_per_sq_foot#275, feb_sales_per_sq_foot#276, mar_sales_per_sq_foot#277, apr_sales_per_sq_foot#278, may_sales_per_sq_foot#279, jun_sales_per_sq_foot#280, jul_sales_per_sq_foot#281, aug_sales_per_sq_foot#282, sep_sales_per_sq_foot#283, oct_sales_per_sq_foot#284, nov_sales_per_sq_foot#285, dec_sales_per_sq_foot#286, jan_net#287, feb_net#288, mar_net#289, apr_net#290, may_net#291, jun_net#292, jul_net#293, aug_net#294, sep_net#295, oct_net#296, nov_net#297, dec_net#298] + +(50) ColumnarToRow [codegen id : 1] +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#139, year#140, jan_sales#263, feb_sales#264, mar_sales#265, apr_sales#266, may_sales#267, jun_sales#268, jul_sales#269, aug_sales#270, sep_sales#271, oct_sales#272, nov_sales#273, dec_sales#274, jan_sales_per_sq_foot#275, feb_sales_per_sq_foot#276, mar_sales_per_sq_foot#277, apr_sales_per_sq_foot#278, may_sales_per_sq_foot#279, jun_sales_per_sq_foot#280, jul_sales_per_sq_foot#281, aug_sales_per_sq_foot#282, sep_sales_per_sq_foot#283, oct_sales_per_sq_foot#284, nov_sales_per_sq_foot#285, dec_sales_per_sq_foot#286, jan_net#287, feb_net#288, mar_net#289, apr_net#290, may_net#291, jun_net#292, jul_net#293, aug_net#294, sep_net#295, oct_net#296, nov_net#297, dec_net#298] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (55) -+- * ColumnarToRow (54) - +- CometFilter (53) - +- CometScan parquet spark_catalog.default.date_dim (52) +BroadcastExchange (54) ++- * ColumnarToRow (53) + +- CometFilter (52) + +- CometScan parquet spark_catalog.default.date_dim (51) -(52) Scan parquet spark_catalog.default.date_dim +(51) 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 -(53) CometFilter +(52) 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)) -(54) ColumnarToRow [codegen id : 1] +(53) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#16, d_year#17, d_moy#18] -(55) BroadcastExchange +(54) 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=4] -Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#127 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 28 Hosting Expression = cs_sold_date_sk#77 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 72133811b..ce4354bc5 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,71 +1,60 @@ -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 (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 (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 (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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [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,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,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,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,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,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_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,d_date_sk,d_year,d_moy] - 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_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometFilter [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 - CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - 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 [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [t_time_sk] #6 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_time] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] - CometBroadcastExchange [sm_ship_mode_sk] #7 - CometProject [sm_ship_mode_sk] - CometFilter [sm_ship_mode_sk,sm_carrier] - 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] #8 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometHashAggregate [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,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,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,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,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,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_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,d_date_sk,d_year,d_moy] - 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_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometFilter [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] - 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 - 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 +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [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] + CometHashAggregate [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,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,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)] + CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year] #1 + CometHashAggregate [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,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] + CometUnion [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] + CometHashAggregate [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,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)] + CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #2 + CometHashAggregate [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,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,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,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,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,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_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,d_date_sk,d_year,d_moy] + 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_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometFilter [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 + CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + 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 [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [t_time_sk] #6 + CometProject [t_time_sk] + CometFilter [t_time_sk,t_time] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] + CometBroadcastExchange [sm_ship_mode_sk] #7 + CometProject [sm_ship_mode_sk] + CometFilter [sm_ship_mode_sk,sm_carrier] + CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] + CometHashAggregate [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,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)] + CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #8 + CometHashAggregate [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,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,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,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,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,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_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,d_date_sk,d_year,d_moy] + 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_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometFilter [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] + 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 + 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 9301e7b8f..e121b160b 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 @@ -2,11 +2,11 @@ TakeOrderedAndProject (28) +- * Filter (27) +- Window (26) - +- * Sort (25) - +- Exchange (24) - +- * HashAggregate (23) - +- Exchange (22) - +- * ColumnarToRow (21) + +- * ColumnarToRow (25) + +- CometSort (24) + +- CometColumnarExchange (23) + +- CometHashAggregate (22) + +- CometColumnarExchange (21) +- CometHashAggregate (20) +- CometExpand (19) +- CometProject (18) @@ -126,39 +126,37 @@ Input [11]: [ss_quantity#3, ss_sales_price#4, i_category#19, i_class#20, i_brand 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))] -(21) ColumnarToRow [codegen id : 1] +(21) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(23) HashAggregate [codegen id : 2] +(22) CometHashAggregate 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))#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] -(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] +(23) CometColumnarExchange +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#30] +Arguments: hashpartitioning(i_category#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(24) CometSort +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#30] +Arguments: [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#30], [i_category#19 ASC NULLS FIRST, sumsales#30 DESC NULLS LAST] -(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) ColumnarToRow [codegen id : 1] +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#30] (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] +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#30] +Arguments: [rank(sumsales#30) windowspecdefinition(i_category#19, sumsales#30 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#31], [i_category#19], [sumsales#30 DESC NULLS LAST] -(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) Filter [codegen id : 2] +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#30, rk#31] +Condition : (rk#31 <= 100) (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] +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#30, rk#31] +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#30 ASC NULLS FIRST, rk#31 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#30, rk#31] ===== Subqueries ===== 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 c37569802..d54aa1fcb 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,44 +1,40 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (4) + WholeStageCodegen (2) Filter [rk] InputAdapter Window [sumsales,i_category] - WholeStageCodegen (3) - Sort [i_category,sumsales] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [i_category] #1 - 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 (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,sum,isEmpty,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,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] - CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] - CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometFilter [s_store_sk,s_store_id] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometColumnarExchange [i_category] #1 + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,spark_grouping_id,sum,isEmpty,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty,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,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] + CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + 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 53443aedc..c07b9bcb9 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,46 +1,42 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * Project (41) - +- * BroadcastHashJoin Inner BuildRight (40) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * HashAggregate (29) - : : +- Exchange (28) - : : +- * 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) - +- BroadcastExchange (39) - +- * ColumnarToRow (38) - +- CometFilter (37) - +- CometScan parquet spark_catalog.default.customer_address (36) +* ColumnarToRow (38) ++- CometTakeOrderedAndProject (37) + +- CometProject (36) + +- CometBroadcastHashJoin (35) + :- CometProject (33) + : +- CometBroadcastHashJoin (32) + : :- CometHashAggregate (28) + : : +- CometColumnarExchange (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) + : +- CometBroadcastExchange (31) + : +- CometFilter (30) + : +- CometScan parquet spark_catalog.default.customer (29) + +- ReusedExchange (34) (1) Scan parquet spark_catalog.default.store_sales @@ -168,110 +164,88 @@ Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_pri 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))] -(27) ColumnarToRow [codegen id : 1] +(27) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(28) Exchange -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 : 4] +(28) CometHashAggregate 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))#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#31, c_current_addr_sk#32, c_first_name#33, c_last_name#34] +(29) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#24, c_current_addr_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), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(31) CometFilter -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 : 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#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 : 4] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#31] -Join type: Inner -Join condition: None +(30) CometFilter +Input [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#26, c_last_name#27] +Condition : (isnotnull(c_customer_sk#24) AND isnotnull(c_current_addr_sk#25)) -(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] +(31) CometBroadcastExchange +Input [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#26, c_last_name#27] +Arguments: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#26, c_last_name#27] -(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 +(32) CometBroadcastHashJoin +Left output [6]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#28, extended_price#29, list_price#30, extended_tax#31] +Right output [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#26, c_last_name#27] +Arguments: [ss_customer_sk#1], [c_customer_sk#24], Inner, BuildRight -(37) CometFilter -Input [2]: [ca_address_sk#35, ca_city#36] -Condition : (isnotnull(ca_address_sk#35) AND isnotnull(ca_city#36)) +(33) CometProject +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#24, c_current_addr_sk#25, c_first_name#26, c_last_name#27] +Arguments: [ss_ticket_number#5, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_current_addr_sk#25, c_first_name#26, c_last_name#27], [ss_ticket_number#5, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_current_addr_sk#25, c_first_name#26, c_last_name#27] -(38) ColumnarToRow [codegen id : 3] -Input [2]: [ca_address_sk#35, ca_city#36] +(34) ReusedExchange [Reuses operator id: 23] +Output [2]: [ca_address_sk#32, ca_city#33] -(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] +(35) CometBroadcastHashJoin +Left output [8]: [ss_ticket_number#5, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_current_addr_sk#25, c_first_name#26, c_last_name#27] +Right output [2]: [ca_address_sk#32, ca_city#33] +Arguments: [c_current_addr_sk#25], [ca_address_sk#32], Inner, NOT (ca_city#33 = bought_city#28), BuildRight -(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#36 = bought_city#27) +(36) CometProject +Input [10]: [ss_ticket_number#5, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_current_addr_sk#25, c_first_name#26, c_last_name#27, ca_address_sk#32, ca_city#33] +Arguments: [c_last_name#27, c_first_name#26, ca_city#33, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30], [c_last_name#27, c_first_name#26, ca_city#33, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] -(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] +(37) CometTakeOrderedAndProject +Input [8]: [c_last_name#27, c_first_name#26, ca_city#33, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#27 ASC NULLS FIRST,ss_ticket_number#5 ASC NULLS FIRST], output=[c_last_name#27,c_first_name#26,ca_city#33,bought_city#28,ss_ticket_number#5,extended_price#29,extended_tax#31,list_price#30]), [c_last_name#27, c_first_name#26, ca_city#33, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30], 100, [c_last_name#27 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#27, c_first_name#26, ca_city#33, 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] +(38) ColumnarToRow [codegen id : 1] +Input [8]: [c_last_name#27, c_first_name#26, ca_city#33, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (47) -+- * ColumnarToRow (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan parquet spark_catalog.default.date_dim (43) +BroadcastExchange (43) ++- * ColumnarToRow (42) + +- CometProject (41) + +- CometFilter (40) + +- CometScan parquet spark_catalog.default.date_dim (39) -(43) Scan parquet spark_catalog.default.date_dim +(39) 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 -(44) CometFilter +(40) 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)) -(45) CometProject +(41) CometProject Input [3]: [d_date_sk#11, d_year#12, d_dom#13] Arguments: [d_date_sk#11], [d_date_sk#11] -(46) ColumnarToRow [codegen id : 1] +(42) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(47) BroadcastExchange +(43) 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=2] 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 c5f56a69c..426f7c489 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,60 +1,48 @@ -TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_city,extended_price,extended_tax,list_price] - 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] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum,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_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_address_sk,ca_city] - CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,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_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,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_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,d_date_sk] - CometFilter [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] - 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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_city] - CometScan parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange [hd_demo_sk] #5 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [ca_address_sk,ca_city] #6 - CometFilter [ca_address_sk,ca_city] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_city] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] + CometProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] + CometBroadcastHashJoin [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk,ca_city] + CometProject [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name] + CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,bought_city,extended_price,list_price,extended_tax,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometHashAggregate [ss_ticket_number,ss_customer_sk,bought_city,extended_price,list_price,extended_tax,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))] + CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum,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_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_address_sk,ca_city] + CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,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_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,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_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,d_date_sk] + CometFilter [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] + 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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange [s_store_sk] #4 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_city] + CometScan parquet spark_catalog.default.store [s_store_sk,s_city] + CometBroadcastExchange [hd_demo_sk] #5 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange [ca_address_sk,ca_city] #6 + CometFilter [ca_address_sk,ca_city] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] #7 + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + ReusedExchange [ca_address_sk,ca_city] #6 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 b931de292..4991d569f 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,48 +1,50 @@ == Physical Plan == -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) +TakeOrderedAndProject (46) ++- * HashAggregate (45) + +- * ColumnarToRow (44) + +- CometColumnarExchange (43) + +- RowToColumnar (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 @@ -238,50 +240,56 @@ Functions [1]: [partial_count(1)] 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] -(42) Exchange +(42) RowToColumnar 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] -(43) HashAggregate [codegen id : 6] +(43) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=5] + +(44) ColumnarToRow [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] + +(45) 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)#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] -(44) TakeOrderedAndProject +(46) 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 = 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) +BroadcastExchange (51) ++- * ColumnarToRow (50) + +- CometProject (49) + +- CometFilter (48) + +- CometScan parquet spark_catalog.default.date_dim (47) -(45) Scan parquet spark_catalog.default.date_dim +(47) 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 -(46) CometFilter +(48) 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)) -(47) CometProject +(49) CometProject Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(48) ColumnarToRow [codegen id : 1] +(50) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(49) BroadcastExchange +(51) BroadcastExchange Input [1]: [d_date_sk#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-v1_4/q69/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt index c9cc4959b..9e1e3edb6 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,70 +1,72 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cnt1,cnt2,cnt3] 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 (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] - 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,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow + ColumnarToRow + InputAdapter + CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] #1 + RowToColumnar + 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] + 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,cs_ship_customer_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [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] + CometBroadcastExchange [ss_customer_sk] #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,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 InputAdapter - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] - CometFilter [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] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,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 + BroadcastExchange #6 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [cs_ship_customer_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,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 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) + BroadcastExchange #7 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,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 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) + BroadcastExchange #8 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] 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 93faa6420..c8d74de3d 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 @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (30) -+- * HashAggregate (29) - +- Exchange (28) - +- * ColumnarToRow (27) +* ColumnarToRow (30) ++- CometTakeOrderedAndProject (29) + +- CometHashAggregate (28) + +- CometColumnarExchange (27) +- CometHashAggregate (26) +- CometProject (25) +- CometBroadcastHashJoin (24) @@ -156,23 +156,21 @@ Input [5]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i 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))] -(27) ColumnarToRow [codegen id : 1] +(27) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(28) Exchange -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 : 2] +(28) CometHashAggregate 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)#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#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] +(29) CometTakeOrderedAndProject +Input [5]: [i_item_id#17, agg1#29, agg2#30, agg3#31, agg4#32] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#17 ASC NULLS FIRST], output=[i_item_id#17,agg1#29,agg2#30,agg3#31,agg4#32]), [i_item_id#17, agg1#29, agg2#30, agg3#31, agg4#32], 100, [i_item_id#17 ASC NULLS FIRST], [i_item_id#17, agg1#29, agg2#30, agg3#31, agg4#32] + +(30) ColumnarToRow [codegen id : 1] +Input [5]: [i_item_id#17, agg1#29, agg2#30, agg3#31, agg4#32] ===== Subqueries ===== 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 f327d5b06..0725862ca 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,42 +1,40 @@ -TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,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,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,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,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] - CometFilter [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #5 - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [p_promo_sk] #6 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_email,p_channel_event] - CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] + CometHashAggregate [i_item_id,agg1,agg2,agg3,agg4,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))] + CometColumnarExchange [i_item_id] #1 + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,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,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,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,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] + CometProject [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [cd_demo_sk] #3 + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_item_id] #5 + CometFilter [i_item_sk,i_item_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange [p_promo_sk] #6 + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_email,p_channel_event] + 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 6198dca1e..1f56b1034 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,48 +1,52 @@ == Physical Plan == -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) +TakeOrderedAndProject (48) ++- * Project (47) + +- Window (46) + +- * ColumnarToRow (45) + +- CometSort (44) + +- CometColumnarExchange (43) + +- RowToColumnar (42) + +- * HashAggregate (41) + +- * ColumnarToRow (40) + +- CometColumnarExchange (39) + +- RowToColumnar (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) + +- * ColumnarToRow (27) + +- CometSort (26) + +- CometHashAggregate (25) + +- CometColumnarExchange (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 @@ -85,7 +89,7 @@ Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight 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] +(9) ColumnarToRow [codegen id : 4] Input [2]: [ss_store_sk#1, ss_net_profit#2] (10) Scan parquet spark_catalog.default.store @@ -99,7 +103,7 @@ ReadSchema: struct Input [3]: [s_store_sk#7, s_county#8, s_state#9] Condition : isnotnull(s_store_sk#7) -(12) ColumnarToRow [codegen id : 4] +(12) ColumnarToRow [codegen id : 3] Input [3]: [s_store_sk#7, s_county#8, s_state#9] (13) Scan parquet spark_catalog.default.store_sales @@ -155,41 +159,39 @@ 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] +(24) CometColumnarExchange Input [2]: [s_state#15, sum#17] +Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(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] +(25) CometHashAggregate 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 +(26) CometSort +Input [3]: [s_state#15, _w0#18, s_state#15] +Arguments: [s_state#15, _w0#18, s_state#15], [s_state#15 ASC NULLS FIRST, _w0#18 DESC NULLS LAST] + +(27) ColumnarToRow [codegen id : 1] +Input [3]: [s_state#15, _w0#18, s_state#15] (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] +Input [3]: [s_state#15, _w0#18, s_state#15] +Arguments: [rank(_w0#18) windowspecdefinition(s_state#15, _w0#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#19], [s_state#15], [_w0#18 DESC NULLS LAST] -(29) Filter [codegen id : 3] -Input [4]: [s_state#15, _w0#19, s_state#15, ranking#20] -Condition : (ranking#20 <= 5) +(29) Filter [codegen id : 2] +Input [4]: [s_state#15, _w0#18, s_state#15, ranking#19] +Condition : (ranking#19 <= 5) -(30) Project [codegen id : 3] +(30) Project [codegen id : 2] Output [1]: [s_state#15] -Input [4]: [s_state#15, _w0#19, s_state#15, ranking#20] +Input [4]: [s_state#15, _w0#18, s_state#15, ranking#19] (31) BroadcastExchange Input [1]: [s_state#15] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] -(32) BroadcastHashJoin [codegen id : 4] +(32) BroadcastHashJoin [codegen id : 3] Left keys [1]: [s_state#9] Right keys [1]: [s_state#15] Join type: LeftSemi @@ -199,87 +201,99 @@ Join condition: None 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] -(34) BroadcastHashJoin [codegen id : 5] +(34) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_store_sk#1] Right keys [1]: [s_store_sk#7] Join type: Inner Join condition: None -(35) Project [codegen id : 5] +(35) Project [codegen id : 4] 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] -(36) Expand [codegen id : 5] +(36) Expand [codegen id : 4] 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] +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#20, s_county#21, spark_grouping_id#22] -(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] +(37) HashAggregate [codegen id : 4] +Input [4]: [ss_net_profit#2, s_state#20, s_county#21, spark_grouping_id#22] +Keys [3]: [s_state#20, s_county#21, spark_grouping_id#22] 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#23] +Results [4]: [s_state#20, s_county#21, spark_grouping_id#22, sum#24] + +(38) RowToColumnar +Input [4]: [s_state#20, s_county#21, spark_grouping_id#22, sum#24] -(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=4] +(39) CometColumnarExchange +Input [4]: [s_state#20, s_county#21, spark_grouping_id#22, sum#24] +Arguments: hashpartitioning(s_state#20, s_county#21, spark_grouping_id#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(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] +(40) ColumnarToRow [codegen id : 5] +Input [4]: [s_state#20, s_county#21, spark_grouping_id#22, sum#24] + +(41) HashAggregate [codegen id : 5] +Input [4]: [s_state#20, s_county#21, spark_grouping_id#22, sum#24] +Keys [3]: [s_state#20, s_county#21, spark_grouping_id#22] 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))#25] +Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#25,17,2) AS total_sum#26, s_state#20, s_county#21, (cast((shiftright(spark_grouping_id#22, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#22, 0) & 1) as tinyint)) AS lochierarchy#27, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#25,17,2) AS _w0#28, (cast((shiftright(spark_grouping_id#22, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#22, 0) & 1) as tinyint)) AS _w1#29, CASE WHEN (cast((shiftright(spark_grouping_id#22, 0) & 1) as tinyint) = 0) THEN s_state#20 END AS _w2#30] + +(42) RowToColumnar +Input [7]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, _w0#28, _w1#29, _w2#30] + +(43) CometColumnarExchange +Input [7]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, _w0#28, _w1#29, _w2#30] +Arguments: hashpartitioning(_w1#29, _w2#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(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=5] +(44) CometSort +Input [7]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, _w0#28, _w1#29, _w2#30] +Arguments: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, _w0#28, _w1#29, _w2#30], [_w1#29 ASC NULLS FIRST, _w2#30 ASC NULLS FIRST, _w0#28 DESC NULLS LAST] -(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 +(45) ColumnarToRow [codegen id : 6] +Input [7]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, _w0#28, _w1#29, _w2#30] -(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] +(46) Window +Input [7]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, _w0#28, _w1#29, _w2#30] +Arguments: [rank(_w0#28) windowspecdefinition(_w1#29, _w2#30, _w0#28 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#31], [_w1#29, _w2#30], [_w0#28 DESC NULLS LAST] -(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] +(47) Project [codegen id : 7] +Output [5]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, rank_within_parent#31] +Input [8]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, _w0#28, _w1#29, _w2#30, rank_within_parent#31] -(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] +(48) TakeOrderedAndProject +Input [5]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, rank_within_parent#31] +Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN s_state#20 END ASC NULLS FIRST, rank_within_parent#31 ASC NULLS FIRST], [total_sum#26, s_state#20, s_county#21, lochierarchy#27, rank_within_parent#31] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -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 +(49) 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 -(46) CometFilter +(50) 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)) -(47) CometProject +(51) CometProject Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(48) ColumnarToRow [codegen id : 1] +(52) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(49) BroadcastExchange +(53) BroadcastExchange Input [1]: [d_date_sk#5] 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/q70/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt index f95e8d040..81e584f75 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,71 +1,73 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (8) + WholeStageCodegen (7) Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] InputAdapter Window [_w0,_w1,_w2] - WholeStageCodegen (7) - Sort [_w1,_w2,_w0] + WholeStageCodegen (6) + ColumnarToRow InputAdapter - Exchange [_w1,_w2] #1 - 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 (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] - ColumnarToRow - InputAdapter - CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - BroadcastHashJoin [s_state,s_state] + CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] + CometColumnarExchange [_w1,_w2] #1 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 + RowToColumnar + WholeStageCodegen (4) + 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] ColumnarToRow InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + CometProject [ss_store_sk,ss_net_profit] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - BroadcastExchange #6 + BroadcastExchange #5 WholeStageCodegen (3) - Project [s_state] - Filter [ranking] + BroadcastHashJoin [s_state,s_state] + ColumnarToRow InputAdapter - Window [_w0,s_state] - WholeStageCodegen (2) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - InputAdapter - Exchange [s_state] #7 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [s_state,sum,ss_net_profit] - CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] - CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #8 - CometFilter [s_store_sk,s_state] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [d_date_sk] #4 + CometFilter [s_store_sk,s_county,s_state] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + Project [s_state] + Filter [ranking] + InputAdapter + Window [_w0,s_state] + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometSort [s_state,_w0] + CometHashAggregate [s_state,_w0,sum,sum(UnscaledValue(ss_net_profit))] + CometColumnarExchange [s_state] #7 + CometHashAggregate [s_state,sum,ss_net_profit] + CometProject [ss_net_profit,s_state] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] + CometProject [ss_net_profit,ss_sold_date_sk,s_state] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [s_store_sk,s_state] #8 + CometFilter [s_store_sk,s_state] + 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 f2128cfff..53691edc5 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,9 +1,9 @@ == Physical Plan == -* Sort (37) -+- Exchange (36) - +- * HashAggregate (35) - +- Exchange (34) - +- * ColumnarToRow (33) +* ColumnarToRow (37) ++- CometSort (36) + +- CometColumnarExchange (35) + +- CometHashAggregate (34) + +- CometColumnarExchange (33) +- CometHashAggregate (32) +- CometProject (31) +- CometBroadcastHashJoin (30) @@ -192,27 +192,25 @@ 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] +(33) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(34) CometHashAggregate 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] +(35) CometColumnarExchange +Input [5]: [brand_id#39, brand#40, t_hour#35, t_minute#36, ext_price#41] +Arguments: rangepartitioning(ext_price#41 DESC NULLS LAST, brand_id#39 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(36) CometSort +Input [5]: [brand_id#39, brand#40, t_hour#35, t_minute#36, ext_price#41] +Arguments: [brand_id#39, brand#40, t_hour#35, t_minute#36, ext_price#41], [ext_price#41 DESC NULLS LAST, brand_id#39 ASC NULLS FIRST] -(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 +(37) ColumnarToRow [codegen id : 1] +Input [5]: [brand_id#39, brand#40, t_hour#35, t_minute#36, ext_price#41] ===== Subqueries ===== 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 c604a8fa7..cd0700c28 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,53 +1,49 @@ -WholeStageCodegen (3) - Sort [ext_price,brand_id] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [ext_price,brand_id] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum,ext_price] - CometProject [i_brand_id,i_brand,ext_price,t_hour,t_minute] - CometBroadcastHashJoin [i_brand_id,i_brand,ext_price,time_sk,t_time_sk,t_hour,t_minute] - CometProject [i_brand_id,i_brand,ext_price,time_sk] - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_brand,ext_price,sold_item_sk,time_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometUnion [ext_price,sold_item_sk,time_sk] - CometProject [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] - CometFilter [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - 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_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] - CometFilter [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometFilter [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_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 [t_time_sk,t_hour,t_minute] #6 - CometProject [t_time_sk,t_hour,t_minute] - CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] + CometSort [brand_id,brand,t_hour,t_minute,ext_price] + CometColumnarExchange [ext_price,brand_id] #1 + CometHashAggregate [brand_id,brand,t_hour,t_minute,ext_price,i_brand,i_brand_id,sum,sum(UnscaledValue(ext_price))] + CometColumnarExchange [i_brand,i_brand_id,t_hour,t_minute] #2 + CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum,ext_price] + CometProject [i_brand_id,i_brand,ext_price,t_hour,t_minute] + CometBroadcastHashJoin [i_brand_id,i_brand,ext_price,time_sk,t_time_sk,t_hour,t_minute] + CometProject [i_brand_id,i_brand,ext_price,time_sk] + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_brand,ext_price,sold_item_sk,time_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 + CometProject [i_item_sk,i_brand_id,i_brand] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] + CometUnion [ext_price,sold_item_sk,time_sk] + CometProject [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] [ext_price,sold_item_sk,time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_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_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + 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_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_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 [t_time_sk,t_hour,t_minute] #6 + CometProject [t_time_sk,t_hour,t_minute] + CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] + 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 26edd145c..2f6da7d32 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,71 +1,72 @@ == Physical Plan == -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) +* ColumnarToRow (68) ++- CometTakeOrderedAndProject (67) + +- CometHashAggregate (66) + +- CometColumnarExchange (65) + +- CometHashAggregate (64) + +- CometProject (63) + +- CometSortMergeJoin (62) + :- CometSort (56) + : +- CometColumnarExchange (55) + : +- RowToColumnar (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) + +- CometSort (61) + +- CometColumnarExchange (60) + +- CometProject (59) + +- CometFilter (58) + +- CometScan parquet spark_catalog.default.catalog_returns (57) (1) Scan parquet spark_catalog.default.catalog_sales @@ -320,101 +321,99 @@ Join condition: None 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#30] -(54) Exchange +(54) RowToColumnar 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=3] -(55) Sort [codegen id : 4] +(55) CometColumnarExchange 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 +Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(56) Scan parquet spark_catalog.default.catalog_returns +(56) CometSort +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, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST] + +(57) 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 -(57) CometFilter +(58) 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)) -(58) CometProject +(59) 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] -(59) ColumnarToRow [codegen id : 5] +(60) CometColumnarExchange Input [2]: [cr_item_sk#31, cr_order_number#32] +Arguments: hashpartitioning(cr_item_sk#31, cr_order_number#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(60) Exchange +(61) CometSort 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] +Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31 ASC NULLS FIRST, cr_order_number#32 ASC NULLS FIRST] -(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 +(62) CometSortMergeJoin +Left output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +Right output [2]: [cr_item_sk#31, cr_order_number#32] +Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#31, cr_order_number#32], LeftOuter -(62) SortMergeJoin [codegen id : 7] -Left keys [2]: [cs_item_sk#4, cs_order_number#6] -Right keys [2]: [cr_item_sk#31, cr_order_number#32] -Join type: LeftOuter -Join condition: None - -(63) Project [codegen id : 7] -Output [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +(63) CometProject 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] +Arguments: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -(64) HashAggregate [codegen id : 7] +(64) CometHashAggregate 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#34] -Results [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#35] -(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] +(65) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=5] -(66) HashAggregate [codegen id : 8] -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#35] +(66) CometHashAggregate +Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] Functions [1]: [count(1)] -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] -(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] +(67) CometTakeOrderedAndProject +Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#37 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#35,promo#36,total_cnt#37]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37], 100, [total_cnt#37 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#35, promo#36, total_cnt#37] + +(68) ColumnarToRow [codegen id : 4] +Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -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 +(69) 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 -(69) CometFilter +(70) 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)) -(70) CometProject +(71) 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] -(71) ColumnarToRow [codegen id : 1] +(72) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -(72) BroadcastExchange +(73) 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=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 6cba2d0e6..1022e1a90 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,94 +1,86 @@ -TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo] - 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 (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 (4) - Sort [cs_item_sk,cs_order_number] - InputAdapter - Exchange [cs_item_sk,cs_order_number] #2 - 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] - 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 [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,d_date_sk,d_week_seq] - 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_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,d_date_sk,d_date,d_week_seq] - 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_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,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_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,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_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_sk,i_item_desc] - 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 [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,w_warehouse_sk,w_warehouse_name] - 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_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,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometFilter [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] - 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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [i_item_sk,i_item_desc] #6 - CometFilter [i_item_sk,i_item_desc] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange [cd_demo_sk] #7 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange [hd_demo_sk] #8 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange [d_date_sk,d_week_seq] #10 - CometFilter [d_date_sk,d_week_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] +WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] + CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count,count(1)] + CometColumnarExchange [i_item_desc,w_warehouse_name,d_week_seq] #1 + CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] + CometProject [w_warehouse_name,i_item_desc,d_week_seq] + CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] + CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + CometColumnarExchange [cs_item_sk,cs_order_number] #2 + RowToColumnar + 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] + 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 [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,d_date_sk,d_week_seq] + 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_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,d_date_sk,d_date,d_week_seq] + 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_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,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_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,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_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_sk,i_item_desc] + 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 [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,w_warehouse_sk,w_warehouse_name] + 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_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,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometFilter [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] + 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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange [i_item_sk,i_item_desc] #6 + CometFilter [i_item_sk,i_item_desc] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + CometBroadcastExchange [cd_demo_sk] #7 + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + CometBroadcastExchange [hd_demo_sk] #8 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] + CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + CometBroadcastExchange [d_date_sk,d_week_seq] #10 + CometFilter [d_date_sk,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter - BroadcastExchange #12 - WholeStageCodegen (2) + BroadcastExchange #11 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk] - InputAdapter - WholeStageCodegen (6) - Sort [cr_item_sk,cr_order_number] - InputAdapter - Exchange [cr_item_sk,cr_order_number] #13 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk] + CometSort [cr_item_sk,cr_order_number] + CometColumnarExchange [cr_item_sk,cr_order_number] #13 + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] 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 88dcba0fb..6ea426978 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,37 +1,36 @@ == Physical Plan == -* 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) +* ColumnarToRow (32) ++- CometSort (31) + +- CometColumnarExchange (30) + +- CometProject (29) + +- CometBroadcastHashJoin (28) + :- CometFilter (24) + : +- CometHashAggregate (23) + : +- CometColumnarExchange (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) + +- CometBroadcastExchange (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.customer (25) (1) Scan parquet spark_catalog.default.store_sales @@ -135,90 +134,84 @@ 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)] -(22) ColumnarToRow [codegen id : 1] +(22) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(24) HashAggregate [codegen id : 3] +(23) CometHashAggregate 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)#17] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#17 AS cnt#18] -(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)) +(24) CometFilter +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] +Condition : ((cnt#17 >= 1) AND (cnt#17 <= 5)) -(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] +(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] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(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) +(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) -(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] +(27) CometBroadcastExchange +Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Arguments: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -(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] +(28) CometBroadcastHashJoin +Left output [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] +Right output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight -(30) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#19] -Join type: Inner -Join condition: None +(29) CometProject +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] +Arguments: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17], [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -(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] +(30) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] -(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] +(31) CometSort +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, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17], [cnt#17 DESC NULLS LAST] -(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 +(32) ColumnarToRow [codegen id : 1] +Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (38) -+- * ColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.date_dim (34) +BroadcastExchange (37) ++- * ColumnarToRow (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) -(34) Scan parquet spark_catalog.default.date_dim +(33) 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 -(35) CometFilter +(34) 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)) -(36) CometProject +(35) CometProject Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(37) ColumnarToRow [codegen id : 1] +(36) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(38) BroadcastExchange +(37) BroadcastExchange Input [1]: [d_date_sk#7] -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/q73/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt index c91c4cf47..5fb8a197a 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,50 +1,42 @@ -WholeStageCodegen (4) - Sort [cnt] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [cnt] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,count] - CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] - CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] - CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #5 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange [hd_demo_sk] #6 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + CometColumnarExchange [cnt] #1 + CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometFilter [ss_ticket_number,ss_customer_sk,cnt] + CometHashAggregate [ss_ticket_number,ss_customer_sk,cnt,count,count(1)] + CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] + CometProject [ss_customer_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange [s_store_sk] #5 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_county] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county] + CometBroadcastExchange [hd_demo_sk] #6 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] #7 + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] 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 83e2737fd..ecc0fd1ca 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 @@ -1,75 +1,72 @@ == Physical Plan == -TakeOrderedAndProject (71) -+- * Project (70) - +- * BroadcastHashJoin Inner BuildRight (69) - :- * 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) - +- * 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) +* ColumnarToRow (68) ++- CometTakeOrderedAndProject (67) + +- CometProject (66) + +- CometBroadcastHashJoin (65) + :- CometProject (50) + : +- CometBroadcastHashJoin (49) + : :- CometBroadcastHashJoin (33) + : : :- CometFilter (16) + : : : +- CometHashAggregate (15) + : : : +- CometColumnarExchange (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) + : : +- CometBroadcastExchange (32) + : : +- CometHashAggregate (31) + : : +- CometColumnarExchange (30) + : : +- CometHashAggregate (29) + : : +- CometProject (28) + : : +- CometBroadcastHashJoin (27) + : : :- CometProject (23) + : : : +- CometBroadcastHashJoin (22) + : : : :- CometFilter (18) + : : : : +- CometScan parquet spark_catalog.default.customer (17) + : : : +- CometBroadcastExchange (21) + : : : +- CometFilter (20) + : : : +- CometScan parquet spark_catalog.default.store_sales (19) + : : +- CometBroadcastExchange (26) + : : +- CometFilter (25) + : : +- CometScan parquet spark_catalog.default.date_dim (24) + : +- CometBroadcastExchange (48) + : +- CometFilter (47) + : +- CometHashAggregate (46) + : +- CometColumnarExchange (45) + : +- CometHashAggregate (44) + : +- CometProject (43) + : +- CometBroadcastHashJoin (42) + : :- CometProject (40) + : : +- CometBroadcastHashJoin (39) + : : :- CometFilter (35) + : : : +- CometScan parquet spark_catalog.default.customer (34) + : : +- CometBroadcastExchange (38) + : : +- CometFilter (37) + : : +- CometScan parquet spark_catalog.default.web_sales (36) + : +- ReusedExchange (41) + +- CometBroadcastExchange (64) + +- CometHashAggregate (63) + +- CometColumnarExchange (62) + +- CometHashAggregate (61) + +- CometProject (60) + +- CometBroadcastHashJoin (59) + :- CometProject (57) + : +- CometBroadcastHashJoin (56) + : :- CometFilter (52) + : : +- CometScan parquet spark_catalog.default.customer (51) + : +- CometBroadcastExchange (55) + : +- CometFilter (54) + : +- CometScan parquet spark_catalog.default.web_sales (53) + +- ReusedExchange (58) (1) Scan parquet spark_catalog.default.customer @@ -137,337 +134,317 @@ Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_yea 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))] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(16) HashAggregate [codegen id : 8] +(15) CometHashAggregate 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))#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] -(17) Filter [codegen id : 8] -Input [2]: [customer_id#13, year_total#14] -Condition : (isnotnull(year_total#14) AND (year_total#14 > 0.00)) +(16) CometFilter +Input [2]: [customer_id#12, year_total#13] +Condition : (isnotnull(year_total#13) AND (year_total#13 > 0.00)) -(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] +(17) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#14, c_customer_id#15, c_first_name#16, c_last_name#17] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(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)) +(18) CometFilter +Input [4]: [c_customer_sk#14, c_customer_id#15, c_first_name#16, c_last_name#17] +Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_customer_id#15)) -(20) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] +(19) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_customer_sk#18, ss_net_paid#19, ss_sold_date_sk#20] 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#20), dynamicpruningexpression(ss_sold_date_sk#20 IN dynamicpruning#21)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(21) CometFilter -Input [3]: [ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] -Condition : isnotnull(ss_customer_sk#19) +(20) CometFilter +Input [3]: [ss_customer_sk#18, ss_net_paid#19, ss_sold_date_sk#20] +Condition : isnotnull(ss_customer_sk#18) -(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] +(21) CometBroadcastExchange +Input [3]: [ss_customer_sk#18, ss_net_paid#19, ss_sold_date_sk#20] +Arguments: [ss_customer_sk#18, ss_net_paid#19, ss_sold_date_sk#20] -(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 +(22) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#14, c_customer_id#15, c_first_name#16, c_last_name#17] +Right output [3]: [ss_customer_sk#18, ss_net_paid#19, ss_sold_date_sk#20] +Arguments: [c_customer_sk#14], [ss_customer_sk#18], Inner, BuildRight -(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] +(23) CometProject +Input [7]: [c_customer_sk#14, c_customer_id#15, c_first_name#16, c_last_name#17, ss_customer_sk#18, ss_net_paid#19, ss_sold_date_sk#20] +Arguments: [c_customer_id#15, c_first_name#16, c_last_name#17, ss_net_paid#19, ss_sold_date_sk#20], [c_customer_id#15, c_first_name#16, c_last_name#17, ss_net_paid#19, ss_sold_date_sk#20] -(25) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#23, d_year#24] +(24) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#22, d_year#23] 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 -(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 - -(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] +(25) CometFilter +Input [2]: [d_date_sk#22, d_year#23] +Condition : (((isnotnull(d_year#23) AND (d_year#23 = 2002)) AND d_year#23 IN (2001,2002)) AND isnotnull(d_date_sk#22)) + +(26) CometBroadcastExchange +Input [2]: [d_date_sk#22, d_year#23] +Arguments: [d_date_sk#22, d_year#23] + +(27) CometBroadcastHashJoin +Left output [5]: [c_customer_id#15, c_first_name#16, c_last_name#17, ss_net_paid#19, ss_sold_date_sk#20] +Right output [2]: [d_date_sk#22, d_year#23] +Arguments: [ss_sold_date_sk#20], [d_date_sk#22], Inner, BuildRight + +(28) CometProject +Input [7]: [c_customer_id#15, c_first_name#16, c_last_name#17, ss_net_paid#19, ss_sold_date_sk#20, d_date_sk#22, d_year#23] +Arguments: [c_customer_id#15, c_first_name#16, c_last_name#17, ss_net_paid#19, d_year#23], [c_customer_id#15, c_first_name#16, c_last_name#17, ss_net_paid#19, d_year#23] + +(29) CometHashAggregate +Input [5]: [c_customer_id#15, c_first_name#16, c_last_name#17, ss_net_paid#19, d_year#23] +Keys [4]: [c_customer_id#15, c_first_name#16, c_last_name#17, d_year#23] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#19))] + +(30) CometColumnarExchange +Input [5]: [c_customer_id#15, c_first_name#16, c_last_name#17, d_year#23, sum#24] +Arguments: hashpartitioning(c_customer_id#15, c_first_name#16, c_last_name#17, d_year#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(31) CometHashAggregate +Input [5]: [c_customer_id#15, c_first_name#16, c_last_name#17, d_year#23, sum#24] +Keys [4]: [c_customer_id#15, c_first_name#16, c_last_name#17, d_year#23] +Functions [1]: [sum(UnscaledValue(ss_net_paid#19))] + +(32) CometBroadcastExchange +Input [4]: [customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28] +Arguments: [customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28] + +(33) CometBroadcastHashJoin +Left output [2]: [customer_id#12, year_total#13] +Right output [4]: [customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28] +Arguments: [customer_id#12], [customer_id#25], Inner, BuildRight + +(34) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#29, c_customer_id#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_customer_id)] ReadSchema: struct -(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)) +(35) CometFilter +Input [4]: [c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] +Condition : (isnotnull(c_customer_sk#29) AND isnotnull(c_customer_id#30)) -(38) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#34, ws_net_paid#35, ws_sold_date_sk#36] +(36) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#37)] +PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#36)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(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) +(37) CometFilter +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Condition : isnotnull(ws_bill_customer_sk#33) + +(38) CometBroadcastExchange +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Arguments: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] -(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] +(39) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] +Right output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Arguments: [c_customer_sk#29], [ws_bill_customer_sk#33], Inner, BuildRight -(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 +(40) CometProject +Input [7]: [c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32, ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Arguments: [c_customer_id#30, c_first_name#31, c_last_name#32, ws_net_paid#34, ws_sold_date_sk#35], [c_customer_id#30, c_first_name#31, c_last_name#32, ws_net_paid#34, ws_sold_date_sk#35] -(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] +(41) ReusedExchange [Reuses operator id: 10] +Output [2]: [d_date_sk#37, d_year#38] -(43) ReusedExchange [Reuses operator id: 10] -Output [2]: [d_date_sk#38, d_year#39] +(42) CometBroadcastHashJoin +Left output [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, ws_net_paid#34, ws_sold_date_sk#35] +Right output [2]: [d_date_sk#37, d_year#38] +Arguments: [ws_sold_date_sk#35], [d_date_sk#37], Inner, BuildRight -(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 +(43) CometProject +Input [7]: [c_customer_id#30, c_first_name#31, c_last_name#32, ws_net_paid#34, ws_sold_date_sk#35, d_date_sk#37, d_year#38] +Arguments: [c_customer_id#30, c_first_name#31, c_last_name#32, ws_net_paid#34, d_year#38], [c_customer_id#30, c_first_name#31, c_last_name#32, ws_net_paid#34, d_year#38] -(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] +(44) CometHashAggregate +Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, ws_net_paid#34, d_year#38] +Keys [4]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#38] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#34))] + +(45) CometColumnarExchange +Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#38, sum#39] +Arguments: hashpartitioning(c_customer_id#30, c_first_name#31, c_last_name#32, d_year#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (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] - -(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 - -(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] - -(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] +Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#38, sum#39] +Keys [4]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#38] +Functions [1]: [sum(UnscaledValue(ws_net_paid#34))] + +(47) CometFilter +Input [2]: [customer_id#40, year_total#41] +Condition : (isnotnull(year_total#41) AND (year_total#41 > 0.00)) + +(48) CometBroadcastExchange +Input [2]: [customer_id#40, year_total#41] +Arguments: [customer_id#40, year_total#41] + +(49) CometBroadcastHashJoin +Left output [6]: [customer_id#12, year_total#13, customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28] +Right output [2]: [customer_id#40, year_total#41] +Arguments: [customer_id#12], [customer_id#40], Inner, BuildRight + +(50) CometProject +Input [8]: [customer_id#12, year_total#13, customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28, customer_id#40, year_total#41] +Arguments: [customer_id#12, year_total#13, customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28, year_total#41], [customer_id#12, year_total#13, customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28, year_total#41] + +(51) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#42, c_customer_id#43, c_first_name#44, c_last_name#45] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(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)) +(52) CometFilter +Input [4]: [c_customer_sk#42, c_customer_id#43, c_first_name#44, c_last_name#45] +Condition : (isnotnull(c_customer_sk#42) AND isnotnull(c_customer_id#43)) -(56) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] +(53) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_customer_sk#46, ws_net_paid#47, ws_sold_date_sk#48] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#50), dynamicpruningexpression(ws_sold_date_sk#50 IN dynamicpruning#51)] +PartitionFilters: [isnotnull(ws_sold_date_sk#48), dynamicpruningexpression(ws_sold_date_sk#48 IN dynamicpruning#49)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(57) CometFilter -Input [3]: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] -Condition : isnotnull(ws_bill_customer_sk#48) +(54) CometFilter +Input [3]: [ws_bill_customer_sk#46, ws_net_paid#47, ws_sold_date_sk#48] +Condition : isnotnull(ws_bill_customer_sk#46) -(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] +(55) CometBroadcastExchange +Input [3]: [ws_bill_customer_sk#46, ws_net_paid#47, ws_sold_date_sk#48] +Arguments: [ws_bill_customer_sk#46, ws_net_paid#47, ws_sold_date_sk#48] + +(56) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#42, c_customer_id#43, c_first_name#44, c_last_name#45] +Right output [3]: [ws_bill_customer_sk#46, ws_net_paid#47, ws_sold_date_sk#48] +Arguments: [c_customer_sk#42], [ws_bill_customer_sk#46], Inner, BuildRight + +(57) CometProject +Input [7]: [c_customer_sk#42, c_customer_id#43, c_first_name#44, c_last_name#45, ws_bill_customer_sk#46, ws_net_paid#47, ws_sold_date_sk#48] +Arguments: [c_customer_id#43, c_first_name#44, c_last_name#45, ws_net_paid#47, ws_sold_date_sk#48], [c_customer_id#43, c_first_name#44, c_last_name#45, ws_net_paid#47, ws_sold_date_sk#48] + +(58) ReusedExchange [Reuses operator id: 26] +Output [2]: [d_date_sk#50, d_year#51] (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 +Left output [5]: [c_customer_id#43, c_first_name#44, c_last_name#45, ws_net_paid#47, ws_sold_date_sk#48] +Right output [2]: [d_date_sk#50, d_year#51] +Arguments: [ws_sold_date_sk#48], [d_date_sk#50], Inner, BuildRight (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) ReusedExchange [Reuses operator id: 27] -Output [2]: [d_date_sk#52, d_year#53] - -(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) 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) 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) 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#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 : 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#55, year_total#56] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] - -(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#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 : 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#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] +Input [7]: [c_customer_id#43, c_first_name#44, c_last_name#45, ws_net_paid#47, ws_sold_date_sk#48, d_date_sk#50, d_year#51] +Arguments: [c_customer_id#43, c_first_name#44, c_last_name#45, ws_net_paid#47, d_year#51], [c_customer_id#43, c_first_name#44, c_last_name#45, ws_net_paid#47, d_year#51] + +(61) CometHashAggregate +Input [5]: [c_customer_id#43, c_first_name#44, c_last_name#45, ws_net_paid#47, d_year#51] +Keys [4]: [c_customer_id#43, c_first_name#44, c_last_name#45, d_year#51] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#47))] + +(62) CometColumnarExchange +Input [5]: [c_customer_id#43, c_first_name#44, c_last_name#45, d_year#51, sum#52] +Arguments: hashpartitioning(c_customer_id#43, c_first_name#44, c_last_name#45, d_year#51, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(63) CometHashAggregate +Input [5]: [c_customer_id#43, c_first_name#44, c_last_name#45, d_year#51, sum#52] +Keys [4]: [c_customer_id#43, c_first_name#44, c_last_name#45, d_year#51] +Functions [1]: [sum(UnscaledValue(ws_net_paid#47))] + +(64) CometBroadcastExchange +Input [2]: [customer_id#53, year_total#54] +Arguments: [customer_id#53, year_total#54] + +(65) CometBroadcastHashJoin +Left output [7]: [customer_id#12, year_total#13, customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28, year_total#41] +Right output [2]: [customer_id#53, year_total#54] +Arguments: [customer_id#12], [customer_id#53], Inner, (CASE WHEN (year_total#41 > 0.00) THEN (year_total#54 / year_total#41) END > CASE WHEN (year_total#13 > 0.00) THEN (year_total#28 / year_total#13) END), BuildRight + +(66) CometProject +Input [9]: [customer_id#12, year_total#13, customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28, year_total#41, customer_id#53, year_total#54] +Arguments: [customer_id#25, customer_first_name#26, customer_last_name#27], [customer_id#25, customer_first_name#26, customer_last_name#27] + +(67) CometTakeOrderedAndProject +Input [3]: [customer_id#25, customer_first_name#26, customer_last_name#27] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#25 ASC NULLS FIRST,customer_id#25 ASC NULLS FIRST,customer_id#25 ASC NULLS FIRST], output=[customer_id#25,customer_first_name#26,customer_last_name#27]), [customer_id#25, customer_first_name#26, customer_last_name#27], 100, [customer_id#25 ASC NULLS FIRST, customer_id#25 ASC NULLS FIRST, customer_id#25 ASC NULLS FIRST], [customer_id#25, customer_first_name#26, customer_last_name#27] + +(68) ColumnarToRow [codegen id : 1] +Input [3]: [customer_id#25, customer_first_name#26, customer_last_name#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (75) -+- * ColumnarToRow (74) - +- CometFilter (73) - +- CometScan parquet spark_catalog.default.date_dim (72) +BroadcastExchange (72) ++- * ColumnarToRow (71) + +- CometFilter (70) + +- CometScan parquet spark_catalog.default.date_dim (69) -(72) Scan parquet spark_catalog.default.date_dim +(69) 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 -(73) CometFilter +(70) 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)) -(74) ColumnarToRow [codegen id : 1] +(71) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_year#10] -(75) BroadcastExchange +(72) BroadcastExchange Input [2]: [d_date_sk#9, d_year#10] -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=5] -Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#21 IN dynamicpruning#22 -BroadcastExchange (79) -+- * ColumnarToRow (78) - +- CometFilter (77) - +- CometScan parquet spark_catalog.default.date_dim (76) +Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#21 +BroadcastExchange (76) ++- * ColumnarToRow (75) + +- CometFilter (74) + +- CometScan parquet spark_catalog.default.date_dim (73) -(76) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#23, d_year#24] +(73) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#22, d_year#23] 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#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)) +(74) CometFilter +Input [2]: [d_date_sk#22, d_year#23] +Condition : (((isnotnull(d_year#23) AND (d_year#23 = 2002)) AND d_year#23 IN (2001,2002)) AND isnotnull(d_date_sk#22)) -(78) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#23, d_year#24] +(75) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#22, d_year#23] -(79) BroadcastExchange -Input [2]: [d_date_sk#23, d_year#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] +(76) BroadcastExchange +Input [2]: [d_date_sk#22, d_year#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -Subquery:3 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#8 -Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#50 IN dynamicpruning#22 +Subquery:4 Hosting operator id = 53 Hosting Expression = ws_sold_date_sk#48 IN dynamicpruning#21 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 7c2a42ca3..0b288df94 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,106 +1,86 @@ -TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] - 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] - BroadcastHashJoin [customer_id,customer_id] - BroadcastHashJoin [customer_id,customer_id] - Filter [year_total] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_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_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - 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,sum,ss_net_paid] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - 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] #11 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 - CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_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 #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] #14 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 - CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_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 +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] + CometProject [customer_id,customer_first_name,customer_last_name] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total,customer_id,year_total] + CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,customer_id,year_total] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total] + CometFilter [customer_id,year_total] + CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,d_year,sum,sum(UnscaledValue(ss_net_paid))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_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_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,year_total] #5 + CometHashAggregate [customer_id,customer_first_name,customer_last_name,year_total,c_customer_id,c_first_name,c_last_name,d_year,sum,sum(UnscaledValue(ss_net_paid))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #6 + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [customer_id,year_total] #10 + CometFilter [customer_id,year_total] + CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,d_year,sum,sum(UnscaledValue(ws_net_paid))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #11 + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_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 + CometBroadcastExchange [customer_id,year_total] #13 + CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,d_year,sum,sum(UnscaledValue(ws_net_paid))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #14 + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_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 1f736c62c..ca9a5cd59 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,136 +1,129 @@ == Physical Plan == -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) +TakeOrderedAndProject (125) ++- * Project (124) + +- * SortMergeJoin Inner (123) + :- * ColumnarToRow (67) + : +- CometSort (66) + : +- CometColumnarExchange (65) + : +- CometFilter (64) + : +- CometHashAggregate (63) + : +- CometColumnarExchange (62) + : +- CometHashAggregate (61) + : +- CometHashAggregate (60) + : +- CometColumnarExchange (59) + : +- CometHashAggregate (58) + : +- CometUnion (57) + : :- CometProject (22) + : : +- CometSortMergeJoin (21) + : : :- CometSort (15) + : : : +- CometColumnarExchange (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) + : : +- CometSort (20) + : : +- CometColumnarExchange (19) + : : +- CometProject (18) + : : +- CometFilter (17) + : : +- CometScan parquet spark_catalog.default.catalog_returns (16) + : :- CometProject (39) + : : +- CometSortMergeJoin (38) + : : :- CometSort (32) + : : : +- CometColumnarExchange (31) + : : : +- CometProject (30) + : : : +- CometBroadcastHashJoin (29) + : : : :- CometProject (27) + : : : : +- CometBroadcastHashJoin (26) + : : : : :- CometFilter (24) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (23) + : : : : +- ReusedExchange (25) + : : : +- ReusedExchange (28) + : : +- CometSort (37) + : : +- CometColumnarExchange (36) + : : +- CometProject (35) + : : +- CometFilter (34) + : : +- CometScan parquet spark_catalog.default.store_returns (33) + : +- CometProject (56) + : +- CometSortMergeJoin (55) + : :- CometSort (49) + : : +- CometColumnarExchange (48) + : : +- CometProject (47) + : : +- CometBroadcastHashJoin (46) + : : :- CometProject (44) + : : : +- CometBroadcastHashJoin (43) + : : : :- CometFilter (41) + : : : : +- CometScan parquet spark_catalog.default.web_sales (40) + : : : +- ReusedExchange (42) + : : +- ReusedExchange (45) + : +- CometSort (54) + : +- CometColumnarExchange (53) + : +- CometProject (52) + : +- CometFilter (51) + : +- CometScan parquet spark_catalog.default.web_returns (50) + +- * ColumnarToRow (122) + +- CometSort (121) + +- CometColumnarExchange (120) + +- CometFilter (119) + +- CometHashAggregate (118) + +- CometColumnarExchange (117) + +- CometHashAggregate (116) + +- CometHashAggregate (115) + +- CometColumnarExchange (114) + +- CometHashAggregate (113) + +- CometUnion (112) + :- CometProject (83) + : +- CometSortMergeJoin (82) + : :- CometSort (79) + : : +- CometColumnarExchange (78) + : : +- CometProject (77) + : : +- CometBroadcastHashJoin (76) + : : :- CometProject (72) + : : : +- CometBroadcastHashJoin (71) + : : : :- CometFilter (69) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (68) + : : : +- ReusedExchange (70) + : : +- CometBroadcastExchange (75) + : : +- CometFilter (74) + : : +- CometScan parquet spark_catalog.default.date_dim (73) + : +- CometSort (81) + : +- ReusedExchange (80) + :- CometProject (97) + : +- CometSortMergeJoin (96) + : :- CometSort (93) + : : +- CometColumnarExchange (92) + : : +- CometProject (91) + : : +- CometBroadcastHashJoin (90) + : : :- CometProject (88) + : : : +- CometBroadcastHashJoin (87) + : : : :- CometFilter (85) + : : : : +- CometScan parquet spark_catalog.default.store_sales (84) + : : : +- ReusedExchange (86) + : : +- ReusedExchange (89) + : +- CometSort (95) + : +- ReusedExchange (94) + +- CometProject (111) + +- CometSortMergeJoin (110) + :- CometSort (107) + : +- CometColumnarExchange (106) + : +- CometProject (105) + : +- CometBroadcastHashJoin (104) + : :- CometProject (102) + : : +- CometBroadcastHashJoin (101) + : : :- CometFilter (99) + : : : +- CometScan parquet spark_catalog.default.web_sales (98) + : : +- ReusedExchange (100) + : +- ReusedExchange (103) + +- CometSort (109) + +- ReusedExchange (108) (1) Scan parquet spark_catalog.default.catalog_sales @@ -197,54 +190,47 @@ Arguments: [cs_sold_date_sk#5], [d_date_sk#13], Inner, BuildRight 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] +(14) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(15) Exchange +(15) CometSort 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=1] +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_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST] -(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 - -(17) Scan parquet spark_catalog.default.catalog_returns +(16) 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 -(18) CometFilter +(17) 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)) -(19) CometProject +(18) 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] -(20) ColumnarToRow [codegen id : 3] -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] - -(21) Exchange +(19) CometColumnarExchange 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=2] +Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(22) Sort [codegen id : 4] +(20) CometSort 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 +Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST] -(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 +(21) CometSortMergeJoin +Left 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] +Right output [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] +Arguments: [cs_order_number#2, cs_item_sk#1], [cr_order_number#16, cr_item_sk#15], LeftOuter -(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] +(22) CometProject 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] +Arguments: [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], [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] -(25) Scan parquet spark_catalog.default.store_sales +(23) 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 [] @@ -252,82 +238,75 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(26) CometFilter +(24) 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) -(27) ReusedExchange [Reuses operator id: 6] +(25) 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) CometBroadcastHashJoin +(26) 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) CometProject +(27) 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: 11] +(28) ReusedExchange [Reuses operator id: 11] Output [2]: [d_date_sk#33, d_year#34] -(31) CometBroadcastHashJoin +(29) 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) CometProject +(30) 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] +(31) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=3] -(34) Exchange +(32) CometSort 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=3] +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_ticket_number#23 ASC NULLS FIRST, ss_item_sk#22 ASC NULLS FIRST] -(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 - -(36) Scan parquet spark_catalog.default.store_returns +(33) 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 -(37) CometFilter +(34) 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)) -(38) CometProject +(35) 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] -(39) ColumnarToRow [codegen id : 8] +(36) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=4] -(40) Exchange +(37) CometSort 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=4] +Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST] -(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 +(38) CometSortMergeJoin +Left 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] +Right output [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] +Arguments: [ss_ticket_number#23, ss_item_sk#22], [sr_ticket_number#36, sr_item_sk#35], LeftOuter -(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 - -(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] +(39) CometProject 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] +Arguments: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41], [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] -(44) Scan parquet spark_catalog.default.web_sales +(40) 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 [] @@ -335,457 +314,434 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(45) CometFilter +(41) 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) -(46) ReusedExchange [Reuses operator id: 6] +(42) 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) CometBroadcastHashJoin +(43) 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) CometProject +(44) 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: 11] +(45) ReusedExchange [Reuses operator id: 11] Output [2]: [d_date_sk#53, d_year#54] -(50) CometBroadcastHashJoin +(46) 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) CometProject +(47) 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) 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] - -(53) Exchange +(48) CometColumnarExchange 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] +Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(54) Sort [codegen id : 12] +(49) CometSort 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 +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_order_number#43 ASC NULLS FIRST, ws_item_sk#42 ASC NULLS FIRST] -(55) Scan parquet spark_catalog.default.web_returns +(50) 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 -(56) CometFilter +(51) 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)) -(57) CometProject +(52) 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] -(58) ColumnarToRow [codegen id : 13] +(53) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=6] -(59) Exchange +(54) CometSort 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=6] +Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST] -(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 - -(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 +(55) CometSortMergeJoin +Left 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] +Right output [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] +Arguments: [ws_order_number#43, ws_item_sk#42], [wr_order_number#56, wr_item_sk#55], LeftOuter -(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] +(56) CometProject 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] +Arguments: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61], [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] -(63) Union +(57) CometUnion +Child 0 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] +Child 1 Input [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41] +Child 2 Input [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61] -(64) HashAggregate [codegen id : 16] +(58) CometHashAggregate 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] -(65) Exchange +(59) CometColumnarExchange 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=7] +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, CometColumnarShuffle, [plan_id=7] -(66) HashAggregate [codegen id : 17] +(60) CometHashAggregate 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] -(67) HashAggregate [codegen id : 17] +(61) CometHashAggregate 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] -(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=8] +(62) CometColumnarExchange +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] +Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(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] +(63) CometHashAggregate +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] 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] -(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) +(64) CometFilter +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] +Condition : isnotnull(sales_cnt#64) + +(65) CometColumnarExchange +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(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=9] +(66) CometSort +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] +Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65], [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] -(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 +(67) ColumnarToRow [codegen id : 1] +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -(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] +(68) Scan parquet spark_catalog.default.catalog_sales +Output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#74), dynamicpruningexpression(cs_sold_date_sk#74 IN dynamicpruning#75)] +PartitionFilters: [isnotnull(cs_sold_date_sk#70), dynamicpruningexpression(cs_sold_date_sk#70 IN dynamicpruning#71)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(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) +(69) CometFilter +Input [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] +Condition : isnotnull(cs_item_sk#66) -(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] +(70) ReusedExchange [Reuses operator id: 6] +Output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -(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 +(71) CometBroadcastHashJoin +Left output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] +Right output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] +Arguments: [cs_item_sk#66], [i_item_sk#72], Inner, BuildRight -(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] +(72) CometProject +Input [10]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] +Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -(78) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#81, d_year#82] +(73) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#77, d_year#78] 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) 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) CometBroadcastExchange -Input [2]: [d_date_sk#81, d_year#82] -Arguments: [d_date_sk#81, d_year#82] +(74) CometFilter +Input [2]: [d_date_sk#77, d_year#78] +Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) -(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 +(75) CometBroadcastExchange +Input [2]: [d_date_sk#77, d_year#78] +Arguments: [d_date_sk#77, d_year#78] -(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] +(76) CometBroadcastHashJoin +Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] +Right output [2]: [d_date_sk#77, d_year#78] +Arguments: [cs_sold_date_sk#70], [d_date_sk#77], Inner, BuildRight -(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] +(77) CometProject +Input [11]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_date_sk#77, d_year#78] +Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -(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=10] +(78) CometColumnarExchange +Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] +Arguments: hashpartitioning(cs_order_number#67, cs_item_sk#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(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 +(79) CometSort +Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] +Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_order_number#67 ASC NULLS FIRST, cs_item_sk#66 ASC NULLS FIRST] -(86) ReusedExchange [Reuses operator id: 21] -Output [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] +(80) ReusedExchange [Reuses operator id: 19] +Output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -(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 +(81) CometSort +Input [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] +Arguments: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82], [cr_order_number#80 ASC NULLS FIRST, cr_item_sk#79 ASC NULLS FIRST] -(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 +(82) CometSortMergeJoin +Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] +Right output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] +Arguments: [cs_order_number#67, cs_item_sk#66], [cr_order_number#80, cr_item_sk#79], LeftOuter -(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] +(83) CometProject +Input [13]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78, cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] +Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21], [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, (cs_quantity#68 - coalesce(cr_return_quantity#81, 0)) AS sales_cnt#20, (cs_ext_sales_price#69 - coalesce(cr_return_amount#82, 0.00)) AS sales_amt#21] -(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] +(84) Scan parquet spark_catalog.default.store_sales +Output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#91), dynamicpruningexpression(ss_sold_date_sk#91 IN dynamicpruning#92)] +PartitionFilters: [isnotnull(ss_sold_date_sk#87), dynamicpruningexpression(ss_sold_date_sk#87 IN dynamicpruning#88)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(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) - -(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] +(85) CometFilter +Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] +Condition : isnotnull(ss_item_sk#83) -(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 +(86) ReusedExchange [Reuses operator id: 6] +Output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -(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] +(87) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] +Right output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] +Arguments: [ss_item_sk#83], [i_item_sk#89], Inner, BuildRight -(95) ReusedExchange [Reuses operator id: 80] -Output [2]: [d_date_sk#98, d_year#99] +(88) CometProject +Input [10]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] +Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93], [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -(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 +(89) ReusedExchange [Reuses operator id: 75] +Output [2]: [d_date_sk#94, d_year#95] -(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] +(90) CometBroadcastHashJoin +Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] +Right output [2]: [d_date_sk#94, d_year#95] +Arguments: [ss_sold_date_sk#87], [d_date_sk#94], Inner, BuildRight -(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] +(91) CometProject +Input [11]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_date_sk#94, d_year#95] +Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -(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] +(92) CometColumnarExchange +Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] +Arguments: hashpartitioning(ss_ticket_number#84, ss_item_sk#83, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [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 +(93) CometSort +Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] +Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_ticket_number#84 ASC NULLS FIRST, ss_item_sk#83 ASC NULLS FIRST] -(101) ReusedExchange [Reuses operator id: 40] -Output [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] +(94) ReusedExchange [Reuses operator id: 36] +Output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -(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 +(95) CometSort +Input [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] +Arguments: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99], [sr_ticket_number#97 ASC NULLS FIRST, sr_item_sk#96 ASC NULLS FIRST] -(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 +(96) CometSortMergeJoin +Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] +Right output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] +Arguments: [ss_ticket_number#84, ss_item_sk#83], [sr_ticket_number#97, sr_item_sk#96], LeftOuter -(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] +(97) CometProject +Input [13]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95, sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] +Arguments: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41], [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, (ss_quantity#85 - coalesce(sr_return_quantity#98, 0)) AS sales_cnt#40, (ss_ext_sales_price#86 - coalesce(sr_return_amt#99, 0.00)) AS sales_amt#41] -(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] +(98) Scan parquet spark_catalog.default.web_sales +Output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#108), dynamicpruningexpression(ws_sold_date_sk#108 IN dynamicpruning#109)] +PartitionFilters: [isnotnull(ws_sold_date_sk#104), dynamicpruningexpression(ws_sold_date_sk#104 IN dynamicpruning#105)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(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) - -(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] +(99) CometFilter +Input [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] +Condition : isnotnull(ws_item_sk#100) -(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 +(100) ReusedExchange [Reuses operator id: 6] +Output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -(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] +(101) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] +Right output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] +Arguments: [ws_item_sk#100], [i_item_sk#106], Inner, BuildRight -(110) ReusedExchange [Reuses operator id: 80] -Output [2]: [d_date_sk#115, d_year#116] +(102) CometProject +Input [10]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] +Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110], [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -(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 +(103) ReusedExchange [Reuses operator id: 75] +Output [2]: [d_date_sk#111, d_year#112] -(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] +(104) CometBroadcastHashJoin +Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] +Right output [2]: [d_date_sk#111, d_year#112] +Arguments: [ws_sold_date_sk#104], [d_date_sk#111], Inner, BuildRight -(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] +(105) CometProject +Input [11]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_date_sk#111, d_year#112] +Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -(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=12] +(106) CometColumnarExchange +Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] +Arguments: hashpartitioning(ws_order_number#101, ws_item_sk#100, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(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 +(107) CometSort +Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] +Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_order_number#101 ASC NULLS FIRST, ws_item_sk#100 ASC NULLS FIRST] -(116) ReusedExchange [Reuses operator id: 59] -Output [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] +(108) ReusedExchange [Reuses operator id: 53] +Output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -(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 +(109) CometSort +Input [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] +Arguments: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116], [wr_order_number#114 ASC NULLS FIRST, wr_item_sk#113 ASC NULLS FIRST] -(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 +(110) CometSortMergeJoin +Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] +Right output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] +Arguments: [ws_order_number#101, ws_item_sk#100], [wr_order_number#114, wr_item_sk#113], LeftOuter -(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] +(111) CometProject +Input [13]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112, wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] +Arguments: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61], [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, (ws_quantity#102 - coalesce(wr_return_quantity#115, 0)) AS sales_cnt#60, (ws_ext_sales_price#103 - coalesce(wr_return_amt#116, 0.00)) AS sales_amt#61] -(120) Union +(112) CometUnion +Child 0 Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] +Child 1 Input [7]: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41] +Child 2 Input [7]: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61] -(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] +(113) CometHashAggregate +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] +Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 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] -(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=13] +(114) CometColumnarExchange +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] +Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(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] +(115) CometHashAggregate +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] +Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 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] -(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] +(116) CometHashAggregate +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] +Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] 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] -(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=14] +(117) CometColumnarExchange +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] +Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] -(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] +(118) CometHashAggregate +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] +Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] 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] -(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) +(119) CometFilter +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] +Condition : isnotnull(sales_cnt#118) + +(120) CometColumnarExchange +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] +Arguments: hashpartitioning(i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] -(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=15] +(121) CometSort +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] +Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119], [i_brand_id#73 ASC NULLS FIRST, i_class_id#74 ASC NULLS FIRST, i_category_id#75 ASC NULLS FIRST, i_manufact_id#76 ASC NULLS FIRST] -(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 +(122) ColumnarToRow [codegen id : 2] +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -(130) SortMergeJoin [codegen id : 39] +(123) SortMergeJoin [codegen id : 3] 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] +Right keys [4]: [i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] Join type: Inner -Join condition: ((cast(sales_cnt#68 as decimal(17,2)) / cast(sales_cnt#123 as decimal(17,2))) < 0.90000000000000000000) +Join condition: ((cast(sales_cnt#64 as decimal(17,2)) / cast(sales_cnt#118 as decimal(17,2))) < 0.90000000000000000000) -(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] +(124) Project [codegen id : 3] +Output [10]: [d_year#78 AS prev_year#120, d_year#14 AS year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#118 AS prev_yr_cnt#122, sales_cnt#64 AS curr_yr_cnt#123, (sales_cnt#64 - sales_cnt#118) AS sales_cnt_diff#124, (sales_amt#65 - sales_amt#119) AS sales_amt_diff#125] +Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65, d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -(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] +(125) TakeOrderedAndProject +Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] +Arguments: 100, [sales_cnt_diff#124 ASC NULLS FIRST], [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (136) -+- * ColumnarToRow (135) - +- CometFilter (134) - +- CometScan parquet spark_catalog.default.date_dim (133) +BroadcastExchange (129) ++- * ColumnarToRow (128) + +- CometFilter (127) + +- CometScan parquet spark_catalog.default.date_dim (126) -(133) Scan parquet spark_catalog.default.date_dim +(126) 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 -(134) CometFilter +(127) 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)) -(135) ColumnarToRow [codegen id : 1] +(128) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] -(136) BroadcastExchange +(129) BroadcastExchange Input [2]: [d_date_sk#13, d_year#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 -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) +Subquery:4 Hosting operator id = 68 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 +BroadcastExchange (133) ++- * ColumnarToRow (132) + +- CometFilter (131) + +- CometScan parquet spark_catalog.default.date_dim (130) -(137) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#81, d_year#82] +(130) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#77, d_year#78] 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 -(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)) +(131) CometFilter +Input [2]: [d_date_sk#77, d_year#78] +Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) -(139) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#81, d_year#82] +(132) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#77, d_year#78] -(140) BroadcastExchange -Input [2]: [d_date_sk#81, d_year#82] +(133) BroadcastExchange +Input [2]: [d_date_sk#77, d_year#78] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] -Subquery:5 Hosting operator id = 90 Hosting Expression = ss_sold_date_sk#91 IN dynamicpruning#75 +Subquery:5 Hosting operator id = 84 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#71 -Subquery:6 Hosting operator id = 105 Hosting Expression = ws_sold_date_sk#108 IN dynamicpruning#75 +Subquery:6 Hosting operator id = 98 Hosting Expression = ws_sold_date_sk#104 IN dynamicpruning#71 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 27a4dcb67..a942e8071 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,229 +1,150 @@ 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 (39) + WholeStageCodegen (3) 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 (19) - Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - 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 (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 (16) - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - InputAdapter - Union - 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 (2) - Sort [cs_order_number,cs_item_sk] - InputAdapter - Exchange [cs_order_number,cs_item_sk] #4 - 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_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,d_date_sk,d_year] - 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,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_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 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,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 [d_date_sk,d_year] #7 - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - WholeStageCodegen (4) - Sort [cr_order_number,cr_item_sk] - InputAdapter - 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_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_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 (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 (7) - Sort [ss_ticket_number,ss_item_sk] - 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_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,d_date_sk,d_year] - 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,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_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 - 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 (9) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - 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_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_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 (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 (12) - Sort [ws_order_number,ws_item_sk] - 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_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,d_date_sk,d_year] - 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,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_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 - 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 (14) - Sort [wr_order_number,wr_item_sk] - InputAdapter - 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_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 + CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum,sum,sum(sales_cnt),sum(UnscaledValue(sales_amt))] + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum,sales_cnt,sales_amt] + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometSort [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] + CometColumnarExchange [cs_order_number,cs_item_sk] #4 + 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_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,d_date_sk,d_year] + 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,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_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 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,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 [d_date_sk,d_year] #7 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometColumnarExchange [cr_order_number,cr_item_sk] #8 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometSort [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] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #9 + 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_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,d_date_sk,d_year] + 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,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_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 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #7 + CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometColumnarExchange [sr_ticket_number,sr_item_sk] #10 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometSort [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] + CometColumnarExchange [ws_order_number,ws_item_sk] #11 + 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_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,d_date_sk,d_year] + 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,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_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 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #7 + CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometColumnarExchange [wr_order_number,wr_item_sk] #12 + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_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 (38) - Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] + WholeStageCodegen (2) + ColumnarToRow InputAdapter - 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] #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] #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 (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 (21) - Sort [cs_order_number,cs_item_sk] - 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_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,d_date_sk,d_year] - 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,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_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 #17 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometBroadcastExchange [d_date_sk,d_year] #18 - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - WholeStageCodegen (23) - Sort [cr_order_number,cr_item_sk] - InputAdapter - 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 (26) - Sort [ss_ticket_number,ss_item_sk] - 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_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,d_date_sk,d_year] - 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,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_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 - 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 (28) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - 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 (31) - Sort [ws_order_number,ws_item_sk] - 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_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,d_date_sk,d_year] - 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,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_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 - 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 (33) - Sort [wr_order_number,wr_item_sk] - InputAdapter - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #12 + CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 + CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum,sum,sum(sales_cnt),sum(UnscaledValue(sales_amt))] + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum,sales_cnt,sales_amt] + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometSort [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] + CometColumnarExchange [cs_order_number,cs_item_sk] #16 + 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_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,d_date_sk,d_year] + 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,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_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 #17 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + CometBroadcastExchange [d_date_sk,d_year] #18 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #8 + CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometSort [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] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #19 + 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_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,d_date_sk,d_year] + 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,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_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 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #18 + CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #10 + CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometSort [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] + CometColumnarExchange [ws_order_number,ws_item_sk] #20 + 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_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,d_date_sk,d_year] + 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,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_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 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #18 + CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + 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 27a4a8c3f..d1e7938f7 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,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (34) -+- * HashAggregate (33) - +- Exchange (32) - +- * ColumnarToRow (31) +* ColumnarToRow (34) ++- CometTakeOrderedAndProject (33) + +- CometHashAggregate (32) + +- CometColumnarExchange (31) +- CometHashAggregate (30) +- CometUnion (29) :- CometProject (12) @@ -177,21 +177,19 @@ Input [6]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, ext_sales_ 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))] -(31) ColumnarToRow [codegen id : 1] +(31) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(33) HashAggregate [codegen id : 2] +(32) CometHashAggregate 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)#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] -(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] +(33) CometTakeOrderedAndProject +Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#39, sales_amt#40] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[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], output=[channel#10,col_name#11,d_year#8,d_qoy#9,i_category#6,sales_cnt#39,sales_amt#40]), [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#39, sales_amt#40], 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#39, sales_amt#40] + +(34) ColumnarToRow [codegen id : 1] +Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#39, sales_amt#40] 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 e463296d4..a3a571fa1 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,38 +1,36 @@ -TakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [channel,col_name,d_year,d_qoy,i_category,count,sum,ext_sales_price] - CometUnion [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometProject [ss_store_sk,ss_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] - CometProject [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_category] - CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_category] #2 - CometFilter [i_item_sk,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_category] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #3 - CometFilter [d_date_sk,d_year,d_qoy] - 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_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] - CometProject [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_category] - CometFilter [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_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_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] - CometProject [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category] - CometBroadcastHashJoin [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_category] - CometFilter [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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 +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] + CometHashAggregate [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt,count,sum,count(1),sum(UnscaledValue(ext_sales_price))] + CometColumnarExchange [channel,col_name,d_year,d_qoy,i_category] #1 + CometHashAggregate [channel,col_name,d_year,d_qoy,i_category,count,sum,ext_sales_price] + CometUnion [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] + CometProject [ss_store_sk,ss_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] + CometProject [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_category] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange [i_item_sk,i_category] #2 + CometFilter [i_item_sk,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_category] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #3 + CometFilter [d_date_sk,d_year,d_qoy] + 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_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] + CometProject [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_category] + CometFilter [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_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_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] + CometProject [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category] + CometBroadcastHashJoin [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_category] + CometFilter [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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 a61e0be7d..2c2016542 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,90 +1,90 @@ == Physical Plan == 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) + +- * ColumnarToRow (84) + +- CometColumnarExchange (83) + +- RowToColumnar (82) + +- * HashAggregate (81) + +- * Expand (80) + +- Union (79) + :- * ColumnarToRow (31) + : +- CometProject (30) + : +- CometBroadcastHashJoin (29) + : :- CometHashAggregate (16) + : : +- CometColumnarExchange (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) + : +- CometBroadcastExchange (28) + : +- CometHashAggregate (27) + : +- CometColumnarExchange (26) + : +- CometHashAggregate (25) + : +- CometProject (24) + : +- CometBroadcastHashJoin (23) + : :- CometProject (21) + : : +- CometBroadcastHashJoin (20) + : : :- CometFilter (18) + : : : +- CometScan parquet spark_catalog.default.store_returns (17) + : : +- ReusedExchange (19) + : +- ReusedExchange (22) + :- * Project (50) + : +- * BroadcastNestedLoopJoin Inner BuildLeft (49) + : :- BroadcastExchange (40) + : : +- * ColumnarToRow (39) + : : +- CometHashAggregate (38) + : : +- CometColumnarExchange (37) + : : +- CometHashAggregate (36) + : : +- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometScan parquet spark_catalog.default.catalog_sales (32) + : : +- ReusedExchange (33) + : +- * ColumnarToRow (48) + : +- CometHashAggregate (47) + : +- CometColumnarExchange (46) + : +- CometHashAggregate (45) + : +- CometProject (44) + : +- CometBroadcastHashJoin (43) + : :- CometScan parquet spark_catalog.default.catalog_returns (41) + : +- ReusedExchange (42) + +- * ColumnarToRow (78) + +- CometProject (77) + +- CometBroadcastHashJoin (76) + :- CometHashAggregate (63) + : +- CometColumnarExchange (62) + : +- CometHashAggregate (61) + : +- CometProject (60) + : +- CometBroadcastHashJoin (59) + : :- CometProject (55) + : : +- CometBroadcastHashJoin (54) + : : :- CometFilter (52) + : : : +- CometScan parquet spark_catalog.default.web_sales (51) + : : +- ReusedExchange (53) + : +- CometBroadcastExchange (58) + : +- CometFilter (57) + : +- CometScan parquet spark_catalog.default.web_page (56) + +- CometBroadcastExchange (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) + +- CometColumnarExchange (73) + +- CometHashAggregate (72) + +- CometProject (71) + +- CometBroadcastHashJoin (70) + :- CometProject (68) + : +- CometBroadcastHashJoin (67) + : :- CometFilter (65) + : : +- CometScan parquet spark_catalog.default.web_returns (64) + : +- ReusedExchange (66) + +- ReusedExchange (69) (1) Scan parquet spark_catalog.default.store_sales @@ -156,340 +156,326 @@ 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))] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarExchange Input [3]: [s_store_sk#8, sum#9, sum#10] +Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(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] +(16) CometHashAggregate 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))#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] -(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] +(17) Scan parquet spark_catalog.default.store_returns +Output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#18), dynamicpruningexpression(sr_returned_date_sk#18 IN dynamicpruning#19)] +PartitionFilters: [isnotnull(sr_returned_date_sk#14), dynamicpruningexpression(sr_returned_date_sk#14 IN dynamicpruning#15)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(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) - -(20) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#20] - -(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 - -(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] - -(23) ReusedExchange [Reuses operator id: 11] -Output [1]: [s_store_sk#21] - -(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 - -(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] - -(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 +(18) CometFilter +Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] +Condition : isnotnull(sr_store_sk#11) + +(19) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#16] + +(20) CometBroadcastHashJoin +Left output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] +Right output [1]: [d_date_sk#16] +Arguments: [sr_returned_date_sk#14], [d_date_sk#16], Inner, BuildRight + +(21) CometProject +Input [5]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14, d_date_sk#16] +Arguments: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13], [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] + +(22) ReusedExchange [Reuses operator id: 11] +Output [1]: [s_store_sk#17] + +(23) CometBroadcastHashJoin +Left output [3]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] +Right output [1]: [s_store_sk#17] +Arguments: [sr_store_sk#11], [s_store_sk#17], Inner, BuildRight + +(24) CometProject +Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, s_store_sk#17] +Arguments: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17], [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] + +(25) CometHashAggregate +Input [3]: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] +Keys [1]: [s_store_sk#17] +Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#12)), partial_sum(UnscaledValue(sr_net_loss#13))] + +(26) CometColumnarExchange +Input [3]: [s_store_sk#17, sum#18, sum#19] +Arguments: hashpartitioning(s_store_sk#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(27) CometHashAggregate +Input [3]: [s_store_sk#17, sum#18, sum#19] +Keys [1]: [s_store_sk#17] +Functions [2]: [sum(UnscaledValue(sr_return_amt#12)), sum(UnscaledValue(sr_net_loss#13))] + +(28) CometBroadcastExchange +Input [3]: [s_store_sk#17, returns#20, profit_loss#21] +Arguments: [s_store_sk#17, returns#20, profit_loss#21] + +(29) CometBroadcastHashJoin +Left output [3]: [s_store_sk#8, sales#22, profit#23] +Right output [3]: [s_store_sk#17, returns#20, profit_loss#21] +Arguments: [s_store_sk#8], [s_store_sk#17], LeftOuter, BuildRight -(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] +(30) CometProject +Input [6]: [s_store_sk#8, sales#22, profit#23, s_store_sk#17, returns#20, profit_loss#21] +Arguments: [sales#22, returns#24, profit#25, channel#26, id#27], [sales#22, coalesce(returns#20, 0.00) AS returns#24, (profit#23 - coalesce(profit_loss#21, 0.00)) AS profit#25, store channel AS channel#26, s_store_sk#8 AS id#27] -(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] +(31) ColumnarToRow [codegen id : 1] +Input [5]: [sales#22, returns#24, profit#25, channel#26, id#27] + +(32) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#35), dynamicpruningexpression(cs_sold_date_sk#35 IN dynamicpruning#36)] +PartitionFilters: [isnotnull(cs_sold_date_sk#31), dynamicpruningexpression(cs_sold_date_sk#31 IN dynamicpruning#32)] ReadSchema: struct -(34) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#37] +(33) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#33] -(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) CometBroadcastHashJoin +Left output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] +Right output [1]: [d_date_sk#33] +Arguments: [cs_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight -(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) CometProject +Input [5]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31, d_date_sk#33] +Arguments: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30], [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] -(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))] +(36) CometHashAggregate +Input [3]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] +Keys [1]: [cs_call_center_sk#28] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#29)), partial_sum(UnscaledValue(cs_net_profit#30))] -(38) ColumnarToRow [codegen id : 5] -Input [3]: [cs_call_center_sk#32, sum#38, sum#39] +(37) CometColumnarExchange +Input [3]: [cs_call_center_sk#28, sum#34, sum#35] +Arguments: hashpartitioning(cs_call_center_sk#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(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] +(38) CometHashAggregate +Input [3]: [cs_call_center_sk#28, sum#34, sum#35] +Keys [1]: [cs_call_center_sk#28] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#29)), sum(UnscaledValue(cs_net_profit#30))] -(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] +(39) ColumnarToRow [codegen id : 2] +Input [3]: [cs_call_center_sk#28, sales#36, profit#37] -(41) BroadcastExchange -Input [3]: [cs_call_center_sk#32, sales#42, profit#43] -Arguments: IdentityBroadcastMode, [plan_id=5] +(40) BroadcastExchange +Input [3]: [cs_call_center_sk#28, sales#36, profit#37] +Arguments: IdentityBroadcastMode, [plan_id=4] -(42) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#44, cr_net_loss#45, cr_returned_date_sk#46] +(41) Scan parquet spark_catalog.default.catalog_returns +Output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#46), dynamicpruningexpression(cr_returned_date_sk#46 IN dynamicpruning#47)] +PartitionFilters: [isnotnull(cr_returned_date_sk#40), dynamicpruningexpression(cr_returned_date_sk#40 IN dynamicpruning#41)] ReadSchema: struct -(43) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#48] +(42) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#42] -(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) CometBroadcastHashJoin +Left output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] +Right output [1]: [d_date_sk#42] +Arguments: [cr_returned_date_sk#40], [d_date_sk#42], Inner, BuildRight -(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) CometProject +Input [4]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40, d_date_sk#42] +Arguments: [cr_return_amount#38, cr_net_loss#39], [cr_return_amount#38, cr_net_loss#39] -(46) CometHashAggregate -Input [2]: [cr_return_amount#44, cr_net_loss#45] +(45) CometHashAggregate +Input [2]: [cr_return_amount#38, cr_net_loss#39] Keys: [] -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] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#38)), partial_sum(UnscaledValue(cr_net_loss#39))] -(48) Exchange -Input [2]: [sum#49, sum#50] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +(46) CometColumnarExchange +Input [2]: [sum#43, sum#44] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(49) HashAggregate -Input [2]: [sum#49, sum#50] +(47) CometHashAggregate +Input [2]: [sum#43, sum#44] Keys: [] -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] +Functions [2]: [sum(UnscaledValue(cr_return_amount#38)), sum(UnscaledValue(cr_net_loss#39))] -(50) BroadcastNestedLoopJoin [codegen id : 8] +(48) ColumnarToRow +Input [2]: [returns#45, profit_loss#46] + +(49) BroadcastNestedLoopJoin [codegen id : 3] Join type: Inner Join condition: None -(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) Project [codegen id : 3] +Output [5]: [sales#36, returns#45, (profit#37 - profit_loss#46) AS profit#47, catalog channel AS channel#48, cs_call_center_sk#28 AS id#49] +Input [5]: [cs_call_center_sk#28, sales#36, profit#37, returns#45, profit_loss#46] -(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] +(51) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#61), dynamicpruningexpression(ws_sold_date_sk#61 IN dynamicpruning#62)] +PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct -(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) +(52) CometFilter +Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] +Condition : isnotnull(ws_web_page_sk#50) -(54) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#63] +(53) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#55] -(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 +(54) CometBroadcastHashJoin +Left output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] +Right output [1]: [d_date_sk#55] +Arguments: [ws_sold_date_sk#53], [d_date_sk#55], Inner, BuildRight -(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] +(55) CometProject +Input [5]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53, d_date_sk#55] +Arguments: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52], [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] -(57) Scan parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#64] +(56) Scan parquet spark_catalog.default.web_page +Output [1]: [wp_web_page_sk#56] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct -(58) CometFilter -Input [1]: [wp_web_page_sk#64] -Condition : isnotnull(wp_web_page_sk#64) - -(59) CometBroadcastExchange -Input [1]: [wp_web_page_sk#64] -Arguments: [wp_web_page_sk#64] +(57) CometFilter +Input [1]: [wp_web_page_sk#56] +Condition : isnotnull(wp_web_page_sk#56) -(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 +(58) CometBroadcastExchange +Input [1]: [wp_web_page_sk#56] +Arguments: [wp_web_page_sk#56] -(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] +(59) CometBroadcastHashJoin +Left output [3]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] +Right output [1]: [wp_web_page_sk#56] +Arguments: [ws_web_page_sk#50], [wp_web_page_sk#56], Inner, BuildRight -(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))] +(60) CometProject +Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] +Arguments: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56], [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] -(63) ColumnarToRow [codegen id : 9] -Input [3]: [wp_web_page_sk#64, sum#65, sum#66] +(61) CometHashAggregate +Input [3]: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] +Keys [1]: [wp_web_page_sk#56] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#51)), partial_sum(UnscaledValue(ws_net_profit#52))] -(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] +(62) CometColumnarExchange +Input [3]: [wp_web_page_sk#56, sum#57, sum#58] +Arguments: hashpartitioning(wp_web_page_sk#56, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(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] +(63) CometHashAggregate +Input [3]: [wp_web_page_sk#56, sum#57, sum#58] +Keys [1]: [wp_web_page_sk#56] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#51)), sum(UnscaledValue(ws_net_profit#52))] -(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] +(64) Scan parquet spark_catalog.default.web_returns +Output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#74), dynamicpruningexpression(wr_returned_date_sk#74 IN dynamicpruning#75)] +PartitionFilters: [isnotnull(wr_returned_date_sk#62), dynamicpruningexpression(wr_returned_date_sk#62 IN dynamicpruning#63)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct -(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) +(65) CometFilter +Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] +Condition : isnotnull(wr_web_page_sk#59) -(68) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#76] +(66) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#64] -(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 +(67) CometBroadcastHashJoin +Left output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] +Right output [1]: [d_date_sk#64] +Arguments: [wr_returned_date_sk#62], [d_date_sk#64], Inner, BuildRight -(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] +(68) CometProject +Input [5]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62, d_date_sk#64] +Arguments: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61], [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] -(71) ReusedExchange [Reuses operator id: 59] -Output [1]: [wp_web_page_sk#77] +(69) ReusedExchange [Reuses operator id: 58] +Output [1]: [wp_web_page_sk#65] -(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 +(70) CometBroadcastHashJoin +Left output [3]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] +Right output [1]: [wp_web_page_sk#65] +Arguments: [wr_web_page_sk#59], [wp_web_page_sk#65], 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] +(71) CometProject +Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] +Arguments: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65], [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] + +(72) CometHashAggregate +Input [3]: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] +Keys [1]: [wp_web_page_sk#65] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#60)), partial_sum(UnscaledValue(wr_net_loss#61))] + +(73) CometColumnarExchange +Input [3]: [wp_web_page_sk#65, sum#66, sum#67] +Arguments: hashpartitioning(wp_web_page_sk#65, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] (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] +Input [3]: [wp_web_page_sk#65, sum#66, sum#67] +Keys [1]: [wp_web_page_sk#65] +Functions [2]: [sum(UnscaledValue(wr_return_amt#60)), sum(UnscaledValue(wr_net_loss#61))] -(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 +(75) CometBroadcastExchange +Input [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] +Arguments: [wp_web_page_sk#65, returns#68, profit_loss#69] + +(76) CometBroadcastHashJoin +Left output [3]: [wp_web_page_sk#56, sales#70, profit#71] +Right output [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] +Arguments: [wp_web_page_sk#56], [wp_web_page_sk#65], LeftOuter, BuildRight + +(77) CometProject +Input [6]: [wp_web_page_sk#56, sales#70, profit#71, wp_web_page_sk#65, returns#68, profit_loss#69] +Arguments: [sales#70, returns#72, profit#73, channel#74, id#75], [sales#70, coalesce(returns#68, 0.00) AS returns#72, (profit#71 - coalesce(profit_loss#69, 0.00)) AS profit#73, web channel AS channel#74, wp_web_page_sk#56 AS id#75] -(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] +(78) ColumnarToRow [codegen id : 4] +Input [5]: [sales#70, returns#72, profit#73, channel#74, id#75] -(81) Union +(79) Union -(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] +(80) Expand [codegen id : 5] +Input [5]: [sales#22, returns#24, profit#25, channel#26, id#27] +Arguments: [[sales#22, returns#24, profit#25, channel#26, id#27, 0], [sales#22, returns#24, profit#25, channel#26, null, 1], [sales#22, returns#24, profit#25, null, null, 3]], [sales#22, returns#24, profit#25, channel#76, id#77, spark_grouping_id#78] -(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] +(81) HashAggregate [codegen id : 5] +Input [6]: [sales#22, returns#24, profit#25, channel#76, id#77, spark_grouping_id#78] +Keys [3]: [channel#76, id#77, spark_grouping_id#78] +Functions [3]: [partial_sum(sales#22), partial_sum(returns#24), partial_sum(profit#25)] +Aggregate Attributes [6]: [sum#79, isEmpty#80, sum#81, isEmpty#82, sum#83, isEmpty#84] +Results [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] -(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] +(82) RowToColumnar +Input [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] -(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] +(83) CometColumnarExchange +Input [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] +Arguments: hashpartitioning(channel#76, id#77, spark_grouping_id#78, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(84) ColumnarToRow [codegen id : 6] +Input [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] + +(85) HashAggregate [codegen id : 6] +Input [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] +Keys [3]: [channel#76, id#77, spark_grouping_id#78] +Functions [3]: [sum(sales#22), sum(returns#24), sum(profit#25)] +Aggregate Attributes [3]: [sum(sales#22)#91, sum(returns#24)#92, sum(profit#25)#93] +Results [5]: [channel#76, id#77, sum(sales#22)#91 AS sales#94, sum(returns#24)#92 AS returns#95, sum(profit#25)#93 AS profit#96] (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] +Input [5]: [channel#76, id#77, sales#94, returns#95, profit#96] +Arguments: 100, [channel#76 ASC NULLS FIRST, id#77 ASC NULLS FIRST], [channel#76, id#77, sales#94, returns#95, profit#96] ===== Subqueries ===== @@ -521,16 +507,16 @@ Input [1]: [d_date_sk#6] (91) BroadcastExchange Input [1]: [d_date_sk#6] -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=9] -Subquery:2 Hosting operator id = 18 Hosting Expression = sr_returned_date_sk#18 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 17 Hosting Expression = sr_returned_date_sk#14 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#35 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#31 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 42 Hosting Expression = cr_returned_date_sk#46 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 41 Hosting Expression = cr_returned_date_sk#40 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 52 Hosting Expression = ws_sold_date_sk#61 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 51 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 66 Hosting Expression = wr_returned_date_sk#74 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 64 Hosting Expression = wr_returned_date_sk#62 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 f9088f784..605180016 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,22 +1,22 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (14) + WholeStageCodegen (6) 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) - 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) - 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel,id,spark_grouping_id] #1 + RowToColumnar + WholeStageCodegen (5) + 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 (1) + ColumnarToRow InputAdapter - Exchange [s_store_sk] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + CometProject [returns,profit,profit_loss,s_store_sk] [sales,returns,profit,channel,id] + CometBroadcastHashJoin [s_store_sk,sales,profit,s_store_sk,returns,profit_loss] + CometHashAggregate [s_store_sk,sales,profit,sum,sum,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit))] + CometColumnarExchange [s_store_sk] #2 CometHashAggregate [s_store_sk,sum,sum,ss_ext_sales_price,ss_net_profit] CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] @@ -39,64 +39,52 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometBroadcastExchange [s_store_sk] #5 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] - InputAdapter - 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] #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometHashAggregate [s_store_sk,sum,sum,sr_return_amt,sr_net_loss] - CometProject [sr_return_amt,sr_net_loss,s_store_sk] - CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] - CometProject [sr_store_sk,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_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 - 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 #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] #9 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter + CometBroadcastExchange [s_store_sk,returns,profit_loss] #6 + CometHashAggregate [s_store_sk,returns,profit_loss,sum,sum,sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss))] + CometColumnarExchange [s_store_sk] #7 + CometHashAggregate [s_store_sk,sum,sum,sr_return_amt,sr_net_loss] + CometProject [sr_return_amt,sr_net_loss,s_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] + CometProject [sr_store_sk,sr_return_amt,sr_net_loss] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_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 + ReusedExchange [d_date_sk] #4 + ReusedExchange [s_store_sk] #5 + WholeStageCodegen (3) + Project [sales,returns,profit,profit_loss,cs_call_center_sk] + BroadcastNestedLoopJoin + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometHashAggregate [cs_call_center_sk,sales,profit,sum,sum,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] + CometColumnarExchange [cs_call_center_sk] #9 CometHashAggregate [cs_call_center_sk,sum,sum,cs_ext_sales_price,cs_net_profit] CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,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 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 #10 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometHashAggregate [returns,profit_loss,sum,sum,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] + CometColumnarExchange #10 CometHashAggregate [sum,sum,cr_return_amount,cr_net_loss] CometProject [cr_return_amount,cr_net_loss] CometBroadcastHashJoin [cr_return_amount,cr_net_loss,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 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] + WholeStageCodegen (4) + ColumnarToRow InputAdapter - Exchange [wp_web_page_sk] #11 - WholeStageCodegen (9) - ColumnarToRow - InputAdapter + CometProject [returns,profit,profit_loss,wp_web_page_sk] [sales,returns,profit,channel,id] + CometBroadcastHashJoin [wp_web_page_sk,sales,profit,wp_web_page_sk,returns,profit_loss] + CometHashAggregate [wp_web_page_sk,sales,profit,sum,sum,sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit))] + CometColumnarExchange [wp_web_page_sk] #11 CometHashAggregate [wp_web_page_sk,sum,sum,ws_ext_sales_price,ws_net_profit] CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] @@ -109,22 +97,16 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometBroadcastExchange [wp_web_page_sk] #12 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] - InputAdapter - 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] #14 - WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometHashAggregate [wp_web_page_sk,sum,sum,wr_return_amt,wr_net_loss] - CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] - CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] - CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk] - CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_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 - ReusedExchange [d_date_sk] #4 - ReusedExchange [wp_web_page_sk] #12 + CometBroadcastExchange [wp_web_page_sk,returns,profit_loss] #13 + CometHashAggregate [wp_web_page_sk,returns,profit_loss,sum,sum,sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss))] + CometColumnarExchange [wp_web_page_sk] #14 + CometHashAggregate [wp_web_page_sk,sum,sum,wr_return_amt,wr_net_loss] + CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] + CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk] + CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_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 + 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/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt index 7f2688112..2ffeb0737 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt @@ -1,74 +1,71 @@ == Physical Plan == -TakeOrderedAndProject (70) -+- * Project (69) - +- * SortMergeJoin Inner (68) - :- * Project (45) - : +- * SortMergeJoin Inner (44) - : :- * Sort (21) - : : +- * HashAggregate (20) - : : +- Exchange (19) - : : +- * HashAggregate (18) - : : +- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Project (14) - : : : +- * Filter (13) - : : : +- * SortMergeJoin LeftOuter (12) - : : : :- * Sort (5) - : : : : +- Exchange (4) - : : : : +- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- * Sort (11) - : : : +- Exchange (10) - : : : +- * ColumnarToRow (9) - : : : +- CometProject (8) - : : : +- CometFilter (7) - : : : +- CometScan parquet spark_catalog.default.store_returns (6) - : : +- ReusedExchange (15) - : +- * Sort (43) - : +- * Filter (42) - : +- * HashAggregate (41) - : +- Exchange (40) - : +- * HashAggregate (39) - : +- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * Project (35) - : : +- * Filter (34) - : : +- * SortMergeJoin LeftOuter (33) - : : :- * Sort (26) - : : : +- Exchange (25) - : : : +- * ColumnarToRow (24) - : : : +- CometFilter (23) - : : : +- CometScan parquet spark_catalog.default.web_sales (22) - : : +- * Sort (32) - : : +- Exchange (31) - : : +- * ColumnarToRow (30) - : : +- CometProject (29) - : : +- CometFilter (28) - : : +- CometScan parquet spark_catalog.default.web_returns (27) - : +- ReusedExchange (36) - +- * Sort (67) - +- * Filter (66) - +- * HashAggregate (65) - +- Exchange (64) - +- * HashAggregate (63) - +- * Project (62) - +- * BroadcastHashJoin Inner BuildRight (61) - :- * Project (59) - : +- * Filter (58) - : +- * SortMergeJoin LeftOuter (57) - : :- * Sort (50) - : : +- Exchange (49) - : : +- * ColumnarToRow (48) - : : +- CometFilter (47) - : : +- CometScan parquet spark_catalog.default.catalog_sales (46) - : +- * Sort (56) - : +- Exchange (55) - : +- * ColumnarToRow (54) - : +- CometProject (53) - : +- CometFilter (52) - : +- CometScan parquet spark_catalog.default.catalog_returns (51) - +- ReusedExchange (60) +TakeOrderedAndProject (67) ++- * Project (66) + +- * ColumnarToRow (65) + +- CometSortMergeJoin (64) + :- CometProject (43) + : +- CometSortMergeJoin (42) + : :- CometSort (21) + : : +- CometHashAggregate (20) + : : +- CometColumnarExchange (19) + : : +- CometHashAggregate (18) + : : +- CometProject (17) + : : +- CometBroadcastHashJoin (16) + : : :- CometProject (12) + : : : +- CometFilter (11) + : : : +- CometSortMergeJoin (10) + : : : :- CometSort (4) + : : : : +- CometColumnarExchange (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometSort (9) + : : : +- CometColumnarExchange (8) + : : : +- CometProject (7) + : : : +- CometFilter (6) + : : : +- CometScan parquet spark_catalog.default.store_returns (5) + : : +- CometBroadcastExchange (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.date_dim (13) + : +- CometSort (41) + : +- CometFilter (40) + : +- CometHashAggregate (39) + : +- CometColumnarExchange (38) + : +- CometHashAggregate (37) + : +- CometProject (36) + : +- CometBroadcastHashJoin (35) + : :- CometProject (33) + : : +- CometFilter (32) + : : +- CometSortMergeJoin (31) + : : :- CometSort (25) + : : : +- CometColumnarExchange (24) + : : : +- CometFilter (23) + : : : +- CometScan parquet spark_catalog.default.web_sales (22) + : : +- CometSort (30) + : : +- CometColumnarExchange (29) + : : +- CometProject (28) + : : +- CometFilter (27) + : : +- CometScan parquet spark_catalog.default.web_returns (26) + : +- ReusedExchange (34) + +- CometSort (63) + +- CometFilter (62) + +- CometHashAggregate (61) + +- CometColumnarExchange (60) + +- CometHashAggregate (59) + +- CometProject (58) + +- CometBroadcastHashJoin (57) + :- CometProject (55) + : +- CometFilter (54) + : +- CometSortMergeJoin (53) + : :- CometSort (47) + : : +- CometColumnarExchange (46) + : : +- CometFilter (45) + : : +- CometScan parquet spark_catalog.default.catalog_sales (44) + : +- CometSort (52) + : +- CometColumnarExchange (51) + : +- CometProject (50) + : +- CometFilter (49) + : +- CometScan parquet spark_catalog.default.catalog_returns (48) + +- ReusedExchange (56) (1) Scan parquet spark_catalog.default.store_sales @@ -83,349 +80,326 @@ ReadSchema: struct -(7) CometFilter +(6) CometFilter Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] Condition : (isnotnull(sr_ticket_number#10) AND isnotnull(sr_item_sk#9)) -(8) CometProject +(7) CometProject Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_item_sk#9, sr_ticket_number#10] -(9) ColumnarToRow [codegen id : 3] -Input [2]: [sr_item_sk#9, sr_ticket_number#10] - -(10) Exchange +(8) CometColumnarExchange Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(11) Sort [codegen id : 4] +(9) CometSort Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST], false, 0 +Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST] -(12) SortMergeJoin [codegen id : 6] -Left keys [2]: [ss_ticket_number#3, ss_item_sk#1] -Right keys [2]: [sr_ticket_number#10, sr_item_sk#9] -Join type: LeftOuter -Join condition: None +(10) CometSortMergeJoin +Left output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] +Right output [2]: [sr_item_sk#9, sr_ticket_number#10] +Arguments: [ss_ticket_number#3, ss_item_sk#1], [sr_ticket_number#10, sr_item_sk#9], LeftOuter -(13) Filter [codegen id : 6] +(11) CometFilter Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] Condition : isnull(sr_ticket_number#10) -(14) Project [codegen id : 6] -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] +(12) CometProject Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -(15) ReusedExchange [Reuses operator id: 74] +(13) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#12, d_year#13] +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 + +(14) CometFilter +Input [2]: [d_date_sk#12, d_year#13] +Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) + +(15) CometBroadcastExchange +Input [2]: [d_date_sk#12, d_year#13] +Arguments: [d_date_sk#12, d_year#13] -(16) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] +Right output [2]: [d_date_sk#12, d_year#13] +Arguments: [ss_sold_date_sk#7], [d_date_sk#12], Inner, BuildRight -(17) Project [codegen id : 6] -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] +(17) CometProject Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] -(18) HashAggregate [codegen id : 6] +(18) CometHashAggregate Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesale_cost#5)), partial_sum(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [3]: [sum#14, sum#15, sum#16] -Results [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] -(19) Exchange -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] -Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(19) CometColumnarExchange +Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] +Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(20) HashAggregate [codegen id : 7] -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] +(20) CometHashAggregate +Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [3]: [sum(ss_quantity#4)#20, sum(UnscaledValue(ss_wholesale_cost#5))#21, sum(UnscaledValue(ss_sales_price#6))#22] -Results [6]: [d_year#13 AS ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, sum(ss_quantity#4)#20 AS ss_qty#24, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#5))#21,17,2) AS ss_wc#25, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#22,17,2) AS ss_sp#26] -(21) Sort [codegen id : 7] -Input [6]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] -Arguments: [ss_sold_year#23 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], false, 0 +(21) CometSort +Input [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] +Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20], [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST] (22) Scan parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] 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#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#28)] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (23) CometFilter -Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Condition : (isnotnull(ws_item_sk#27) AND isnotnull(ws_bill_customer_sk#28)) +Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] +Condition : (isnotnull(ws_item_sk#21) AND isnotnull(ws_bill_customer_sk#22)) -(24) ColumnarToRow [codegen id : 8] -Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +(24) CometColumnarExchange +Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] +Arguments: hashpartitioning(ws_order_number#23, ws_item_sk#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(25) Exchange -Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Arguments: hashpartitioning(ws_order_number#29, ws_item_sk#27, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(25) CometSort +Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] +Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_order_number#23 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST] -(26) Sort [codegen id : 9] -Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Arguments: [ws_order_number#29 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST], false, 0 - -(27) Scan parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] +(26) Scan parquet spark_catalog.default.web_returns +Output [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct -(28) CometFilter -Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] -Condition : (isnotnull(wr_order_number#36) AND isnotnull(wr_item_sk#35)) - -(29) CometProject -Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] -Arguments: [wr_item_sk#35, wr_order_number#36], [wr_item_sk#35, wr_order_number#36] - -(30) ColumnarToRow [codegen id : 10] -Input [2]: [wr_item_sk#35, wr_order_number#36] - -(31) Exchange -Input [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: hashpartitioning(wr_order_number#36, wr_item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(32) Sort [codegen id : 11] -Input [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: [wr_order_number#36 ASC NULLS FIRST, wr_item_sk#35 ASC NULLS FIRST], false, 0 - -(33) SortMergeJoin [codegen id : 13] -Left keys [2]: [ws_order_number#29, ws_item_sk#27] -Right keys [2]: [wr_order_number#36, wr_item_sk#35] -Join type: LeftOuter -Join condition: None - -(34) Filter [codegen id : 13] -Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] -Condition : isnull(wr_order_number#36) - -(35) Project [codegen id : 13] -Output [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] - -(36) ReusedExchange [Reuses operator id: 74] -Output [2]: [d_date_sk#38, d_year#39] - -(37) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#38] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 13] -Output [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] -Input [8]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, d_date_sk#38, d_year#39] - -(39) HashAggregate [codegen id : 13] -Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] -Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] -Functions [3]: [partial_sum(ws_quantity#30), partial_sum(UnscaledValue(ws_wholesale_cost#31)), partial_sum(UnscaledValue(ws_sales_price#32))] -Aggregate Attributes [3]: [sum#40, sum#41, sum#42] -Results [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] - -(40) Exchange -Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] -Arguments: hashpartitioning(d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, 5), ENSURE_REQUIREMENTS, [plan_id=6] - -(41) HashAggregate [codegen id : 14] -Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] -Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] -Functions [3]: [sum(ws_quantity#30), sum(UnscaledValue(ws_wholesale_cost#31)), sum(UnscaledValue(ws_sales_price#32))] -Aggregate Attributes [3]: [sum(ws_quantity#30)#46, sum(UnscaledValue(ws_wholesale_cost#31))#47, sum(UnscaledValue(ws_sales_price#32))#48] -Results [6]: [d_year#39 AS ws_sold_year#49, ws_item_sk#27, ws_bill_customer_sk#28 AS ws_customer_sk#50, sum(ws_quantity#30)#46 AS ws_qty#51, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#31))#47,17,2) AS ws_wc#52, MakeDecimal(sum(UnscaledValue(ws_sales_price#32))#48,17,2) AS ws_sp#53] - -(42) Filter [codegen id : 14] -Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] -Condition : (coalesce(ws_qty#51, 0) > 0) - -(43) Sort [codegen id : 14] -Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] -Arguments: [ws_sold_year#49 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST, ws_customer_sk#50 ASC NULLS FIRST], false, 0 - -(44) SortMergeJoin [codegen id : 15] -Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50] -Join type: Inner -Join condition: None - -(45) Project [codegen id : 15] -Output [9]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53] -Input [12]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] - -(46) Scan parquet spark_catalog.default.catalog_sales -Output [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +(27) CometFilter +Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] +Condition : (isnotnull(wr_order_number#30) AND isnotnull(wr_item_sk#29)) + +(28) CometProject +Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] +Arguments: [wr_item_sk#29, wr_order_number#30], [wr_item_sk#29, wr_order_number#30] + +(29) CometColumnarExchange +Input [2]: [wr_item_sk#29, wr_order_number#30] +Arguments: hashpartitioning(wr_order_number#30, wr_item_sk#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(30) CometSort +Input [2]: [wr_item_sk#29, wr_order_number#30] +Arguments: [wr_item_sk#29, wr_order_number#30], [wr_order_number#30 ASC NULLS FIRST, wr_item_sk#29 ASC NULLS FIRST] + +(31) CometSortMergeJoin +Left output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] +Right output [2]: [wr_item_sk#29, wr_order_number#30] +Arguments: [ws_order_number#23, ws_item_sk#21], [wr_order_number#30, wr_item_sk#29], LeftOuter + +(32) CometFilter +Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] +Condition : isnull(wr_order_number#30) + +(33) CometProject +Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] +Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] + +(34) ReusedExchange [Reuses operator id: 15] +Output [2]: [d_date_sk#32, d_year#33] + +(35) CometBroadcastHashJoin +Left output [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] +Right output [2]: [d_date_sk#32, d_year#33] +Arguments: [ws_sold_date_sk#27], [d_date_sk#32], Inner, BuildRight + +(36) CometProject +Input [8]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, d_date_sk#32, d_year#33] +Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] + +(37) CometHashAggregate +Input [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] +Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] +Functions [3]: [partial_sum(ws_quantity#24), partial_sum(UnscaledValue(ws_wholesale_cost#25)), partial_sum(UnscaledValue(ws_sales_price#26))] + +(38) CometColumnarExchange +Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] +Arguments: hashpartitioning(d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(39) CometHashAggregate +Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] +Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] +Functions [3]: [sum(ws_quantity#24), sum(UnscaledValue(ws_wholesale_cost#25)), sum(UnscaledValue(ws_sales_price#26))] + +(40) CometFilter +Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] +Condition : (coalesce(ws_qty#39, 0) > 0) + +(41) CometSort +Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] +Arguments: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41], [ws_sold_year#37 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST, ws_customer_sk#38 ASC NULLS FIRST] + +(42) CometSortMergeJoin +Left output [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] +Right output [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] +Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38], Inner + +(43) CometProject +Input [12]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] +Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41], [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] + +(44) Scan parquet spark_catalog.default.catalog_sales +Output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_sold_date_sk#60 IN dynamicpruning#61)] +PartitionFilters: [isnotnull(cs_sold_date_sk#48), dynamicpruningexpression(cs_sold_date_sk#48 IN dynamicpruning#49)] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(47) CometFilter -Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Condition : (isnotnull(cs_item_sk#55) AND isnotnull(cs_bill_customer_sk#54)) - -(48) ColumnarToRow [codegen id : 16] -Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +(45) CometFilter +Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] +Condition : (isnotnull(cs_item_sk#43) AND isnotnull(cs_bill_customer_sk#42)) -(49) Exchange -Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Arguments: hashpartitioning(cs_order_number#56, cs_item_sk#55, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(46) CometColumnarExchange +Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] +Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(50) Sort [codegen id : 17] -Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Arguments: [cs_order_number#56 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST], false, 0 +(47) CometSort +Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] +Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_order_number#44 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST] -(51) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] +(48) Scan parquet spark_catalog.default.catalog_returns +Output [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(52) CometFilter -Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] -Condition : (isnotnull(cr_order_number#63) AND isnotnull(cr_item_sk#62)) - -(53) CometProject -Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] -Arguments: [cr_item_sk#62, cr_order_number#63], [cr_item_sk#62, cr_order_number#63] - -(54) ColumnarToRow [codegen id : 18] -Input [2]: [cr_item_sk#62, cr_order_number#63] - -(55) Exchange -Input [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: hashpartitioning(cr_order_number#63, cr_item_sk#62, 5), ENSURE_REQUIREMENTS, [plan_id=8] - -(56) Sort [codegen id : 19] -Input [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: [cr_order_number#63 ASC NULLS FIRST, cr_item_sk#62 ASC NULLS FIRST], false, 0 - -(57) SortMergeJoin [codegen id : 21] -Left keys [2]: [cs_order_number#56, cs_item_sk#55] -Right keys [2]: [cr_order_number#63, cr_item_sk#62] -Join type: LeftOuter -Join condition: None - -(58) Filter [codegen id : 21] -Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] -Condition : isnull(cr_order_number#63) - -(59) Project [codegen id : 21] -Output [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] - -(60) ReusedExchange [Reuses operator id: 74] -Output [2]: [d_date_sk#65, d_year#66] - -(61) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [cs_sold_date_sk#60] -Right keys [1]: [d_date_sk#65] -Join type: Inner -Join condition: None - -(62) Project [codegen id : 21] -Output [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] -Input [8]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, d_date_sk#65, d_year#66] - -(63) HashAggregate [codegen id : 21] -Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] -Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] -Functions [3]: [partial_sum(cs_quantity#57), partial_sum(UnscaledValue(cs_wholesale_cost#58)), partial_sum(UnscaledValue(cs_sales_price#59))] -Aggregate Attributes [3]: [sum#67, sum#68, sum#69] -Results [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] - -(64) Exchange -Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] -Arguments: hashpartitioning(d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, 5), ENSURE_REQUIREMENTS, [plan_id=9] - -(65) HashAggregate [codegen id : 22] -Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] -Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] -Functions [3]: [sum(cs_quantity#57), sum(UnscaledValue(cs_wholesale_cost#58)), sum(UnscaledValue(cs_sales_price#59))] -Aggregate Attributes [3]: [sum(cs_quantity#57)#73, sum(UnscaledValue(cs_wholesale_cost#58))#74, sum(UnscaledValue(cs_sales_price#59))#75] -Results [6]: [d_year#66 AS cs_sold_year#76, cs_item_sk#55, cs_bill_customer_sk#54 AS cs_customer_sk#77, sum(cs_quantity#57)#73 AS cs_qty#78, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#58))#74,17,2) AS cs_wc#79, MakeDecimal(sum(UnscaledValue(cs_sales_price#59))#75,17,2) AS cs_sp#80] - -(66) Filter [codegen id : 22] -Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] -Condition : (coalesce(cs_qty#78, 0) > 0) - -(67) Sort [codegen id : 22] -Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] -Arguments: [cs_sold_year#76 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST, cs_customer_sk#77 ASC NULLS FIRST], false, 0 - -(68) SortMergeJoin [codegen id : 23] -Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77] -Join type: Inner -Join condition: None - -(69) Project [codegen id : 23] -Output [12]: [round((cast(ss_qty#24 as double) / cast(coalesce((ws_qty#51 + cs_qty#78), 1) as double)), 2) AS ratio#81, ss_qty#24 AS store_qty#82, ss_wc#25 AS store_wholesale_cost#83, ss_sp#26 AS store_sales_price#84, (coalesce(ws_qty#51, 0) + coalesce(cs_qty#78, 0)) AS other_chan_qty#85, (coalesce(ws_wc#52, 0.00) + coalesce(cs_wc#79, 0.00)) AS other_chan_wholesale_cost#86, (coalesce(ws_sp#53, 0.00) + coalesce(cs_sp#80, 0.00)) AS other_chan_sales_price#87, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, cs_qty#78] -Input [15]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53, cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] - -(70) TakeOrderedAndProject -Input [12]: [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, cs_qty#78] -Arguments: 100, [ratio#81 ASC NULLS FIRST, ss_qty#24 DESC NULLS LAST, ss_wc#25 DESC NULLS LAST, ss_sp#26 DESC NULLS LAST, other_chan_qty#85 ASC NULLS FIRST, other_chan_wholesale_cost#86 ASC NULLS FIRST, other_chan_sales_price#87 ASC NULLS FIRST, round((cast(ss_qty#24 as double) / cast(coalesce((ws_qty#51 + cs_qty#78), 1) as double)), 2) ASC NULLS FIRST], [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87] +(49) CometFilter +Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] +Condition : (isnotnull(cr_order_number#51) AND isnotnull(cr_item_sk#50)) + +(50) CometProject +Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] +Arguments: [cr_item_sk#50, cr_order_number#51], [cr_item_sk#50, cr_order_number#51] + +(51) CometColumnarExchange +Input [2]: [cr_item_sk#50, cr_order_number#51] +Arguments: hashpartitioning(cr_order_number#51, cr_item_sk#50, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(52) CometSort +Input [2]: [cr_item_sk#50, cr_order_number#51] +Arguments: [cr_item_sk#50, cr_order_number#51], [cr_order_number#51 ASC NULLS FIRST, cr_item_sk#50 ASC NULLS FIRST] + +(53) CometSortMergeJoin +Left output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] +Right output [2]: [cr_item_sk#50, cr_order_number#51] +Arguments: [cs_order_number#44, cs_item_sk#43], [cr_order_number#51, cr_item_sk#50], LeftOuter + +(54) CometFilter +Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] +Condition : isnull(cr_order_number#51) + +(55) CometProject +Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] +Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] + +(56) ReusedExchange [Reuses operator id: 15] +Output [2]: [d_date_sk#53, d_year#54] + +(57) CometBroadcastHashJoin +Left output [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] +Right output [2]: [d_date_sk#53, d_year#54] +Arguments: [cs_sold_date_sk#48], [d_date_sk#53], Inner, BuildRight + +(58) CometProject +Input [8]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, d_date_sk#53, d_year#54] +Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] + +(59) CometHashAggregate +Input [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] +Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] +Functions [3]: [partial_sum(cs_quantity#45), partial_sum(UnscaledValue(cs_wholesale_cost#46)), partial_sum(UnscaledValue(cs_sales_price#47))] + +(60) CometColumnarExchange +Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] +Arguments: hashpartitioning(d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(61) CometHashAggregate +Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] +Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] +Functions [3]: [sum(cs_quantity#45), sum(UnscaledValue(cs_wholesale_cost#46)), sum(UnscaledValue(cs_sales_price#47))] + +(62) CometFilter +Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] +Condition : (coalesce(cs_qty#60, 0) > 0) + +(63) CometSort +Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] +Arguments: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62], [cs_sold_year#58 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST, cs_customer_sk#59 ASC NULLS FIRST] + +(64) CometSortMergeJoin +Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] +Right output [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] +Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59], Inner + +(65) ColumnarToRow [codegen id : 1] +Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] + +(66) Project [codegen id : 1] +Output [12]: [round((cast(ss_qty#18 as double) / cast(coalesce((ws_qty#39 + cs_qty#60), 1) as double)), 2) AS ratio#63, ss_qty#18 AS store_qty#64, ss_wc#19 AS store_wholesale_cost#65, ss_sp#20 AS store_sales_price#66, (coalesce(ws_qty#39, 0) + coalesce(cs_qty#60, 0)) AS other_chan_qty#67, (coalesce(ws_wc#40, 0.00) + coalesce(cs_wc#61, 0.00)) AS other_chan_wholesale_cost#68, (coalesce(ws_sp#41, 0.00) + coalesce(cs_sp#62, 0.00)) AS other_chan_sales_price#69, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, cs_qty#60] +Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] + +(67) TakeOrderedAndProject +Input [12]: [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, cs_qty#60] +Arguments: 100, [ratio#63 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 DESC NULLS LAST, ss_sp#20 DESC NULLS LAST, other_chan_qty#67 ASC NULLS FIRST, other_chan_wholesale_cost#68 ASC NULLS FIRST, other_chan_sales_price#69 ASC NULLS FIRST, round((cast(ss_qty#18 as double) / cast(coalesce((ws_qty#39 + cs_qty#60), 1) as double)), 2) ASC NULLS FIRST], [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (74) -+- * ColumnarToRow (73) - +- CometFilter (72) - +- CometScan parquet spark_catalog.default.date_dim (71) +BroadcastExchange (71) ++- * ColumnarToRow (70) + +- CometFilter (69) + +- CometScan parquet spark_catalog.default.date_dim (68) -(71) Scan parquet spark_catalog.default.date_dim +(68) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#12, d_year#13] 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 -(72) CometFilter +(69) CometFilter Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(73) ColumnarToRow [codegen id : 1] +(70) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] -(74) BroadcastExchange +(71) BroadcastExchange Input [2]: [d_date_sk#12, d_year#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#48 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt index 13a8aecff..b7f5911e7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt @@ -1,127 +1,78 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ws_qty,cs_qty,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (23) + WholeStageCodegen (1) Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp] - SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,cs_sold_year,cs_item_sk,cs_customer_sk] + ColumnarToRow InputAdapter - WholeStageCodegen (15) - Project [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] - SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ws_sold_year,ws_item_sk,ws_customer_sk] - InputAdapter - WholeStageCodegen (7) - Sort [ss_sold_year,ss_item_sk,ss_customer_sk] - HashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] [sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price)),ss_sold_year,ss_qty,ss_wc,ss_sp,sum,sum,sum] - InputAdapter - Exchange [d_year,ss_item_sk,ss_customer_sk] #1 - WholeStageCodegen (6) - HashAggregate [d_year,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] [sum,sum,sum,sum,sum,sum] - Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - Filter [sr_ticket_number] - SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - WholeStageCodegen (2) - Sort [ss_ticket_number,ss_item_sk] - InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - WholeStageCodegen (4) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - Exchange [sr_ticket_number,sr_item_sk] #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - WholeStageCodegen (14) - Sort [ws_sold_year,ws_item_sk,ws_customer_sk] - Filter [ws_qty] - HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] [sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price)),ws_sold_year,ws_customer_sk,ws_qty,ws_wc,ws_sp,sum,sum,sum] - InputAdapter - Exchange [d_year,ws_item_sk,ws_bill_customer_sk] #5 - WholeStageCodegen (13) - HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] [sum,sum,sum,sum,sum,sum] - Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - Filter [wr_order_number] - SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - InputAdapter - WholeStageCodegen (9) - Sort [ws_order_number,ws_item_sk] - InputAdapter - Exchange [ws_order_number,ws_item_sk] #6 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - WholeStageCodegen (11) - Sort [wr_order_number,wr_item_sk] - InputAdapter - Exchange [wr_order_number,wr_item_sk] #7 - WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - WholeStageCodegen (22) - Sort [cs_sold_year,cs_item_sk,cs_customer_sk] - Filter [cs_qty] - HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] [sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price)),cs_sold_year,cs_customer_sk,cs_qty,cs_wc,cs_sp,sum,sum,sum] - InputAdapter - Exchange [d_year,cs_item_sk,cs_bill_customer_sk] #8 - WholeStageCodegen (21) - HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] [sum,sum,sum,sum,sum,sum] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - Filter [cr_order_number] - SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - InputAdapter - WholeStageCodegen (17) - Sort [cs_order_number,cs_item_sk] - InputAdapter - Exchange [cs_order_number,cs_item_sk] #9 - WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - WholeStageCodegen (19) - Sort [cr_order_number,cr_item_sk] - InputAdapter - Exchange [cr_order_number,cr_item_sk] #10 - WholeStageCodegen (18) + CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] + CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] + CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] + CometSort [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp] + CometHashAggregate [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,d_year,sum,sum,sum,sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price))] + CometColumnarExchange [d_year,ss_item_sk,ss_customer_sk] #1 + CometHashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum,ss_quantity,ss_wholesale_cost,ss_sales_price] + CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #2 + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometSort [sr_item_sk,sr_ticket_number] + CometColumnarExchange [sr_ticket_number,sr_item_sk] #4 + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #5 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometSort [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] + CometFilter [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] + CometHashAggregate [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp,d_year,ws_bill_customer_sk,sum,sum,sum,sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price))] + CometColumnarExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 + CometHashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum,ws_quantity,ws_wholesale_cost,ws_sales_price] + CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] + CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,d_date_sk,d_year] + CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] + CometSortMergeJoin [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] + CometSort [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometColumnarExchange [ws_order_number,ws_item_sk] #7 + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_item_sk,wr_order_number] + CometColumnarExchange [wr_order_number,wr_item_sk] #8 + CometProject [wr_item_sk,wr_order_number] + CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] + ReusedExchange [d_date_sk,d_year] #5 + CometSort [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] + CometFilter [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] + CometHashAggregate [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp,d_year,cs_bill_customer_sk,sum,sum,sum,sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price))] + CometColumnarExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 + CometHashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum,cs_quantity,cs_wholesale_cost,cs_sales_price] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] + CometSortMergeJoin [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] + CometSort [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + CometColumnarExchange [cs_order_number,cs_item_sk] #10 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [cr_item_sk,cr_order_number] + CometColumnarExchange [cr_order_number,cr_item_sk] #11 + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] + ReusedExchange [d_date_sk,d_year] #5 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 3bad7fc44..4c679d673 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,35 +1,34 @@ == Physical Plan == -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) +* ColumnarToRow (30) ++- CometTakeOrderedAndProject (29) + +- CometProject (28) + +- CometBroadcastHashJoin (27) + :- CometHashAggregate (23) + : +- CometColumnarExchange (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) + +- CometBroadcastExchange (26) + +- CometFilter (25) + +- CometScan parquet spark_catalog.default.customer (24) (1) Scan parquet spark_catalog.default.store_sales @@ -133,82 +132,76 @@ Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, 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))] -(22) ColumnarToRow [codegen id : 1] +(22) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(24) HashAggregate [codegen id : 3] +(23) CometHashAggregate 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#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] -(25) Scan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] +(24) Scan parquet spark_catalog.default.customer +Output [3]: [c_customer_sk#21, c_first_name#22, c_last_name#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(26) CometFilter -Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Condition : isnotnull(c_customer_sk#25) +(25) CometFilter +Input [3]: [c_customer_sk#21, c_first_name#22, c_last_name#23] +Condition : isnotnull(c_customer_sk#21) -(27) ColumnarToRow [codegen id : 2] -Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] +(26) CometBroadcastExchange +Input [3]: [c_customer_sk#21, c_first_name#22, c_last_name#23] +Arguments: [c_customer_sk#21, c_first_name#22, c_last_name#23] -(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=2] +(27) CometBroadcastHashJoin +Left output [5]: [ss_ticket_number#5, ss_customer_sk#1, s_city#15, amt#24, profit#25] +Right output [3]: [c_customer_sk#21, c_first_name#22, c_last_name#23] +Arguments: [ss_customer_sk#1], [c_customer_sk#21], Inner, BuildRight -(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 +(28) CometProject +Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#15, amt#24, profit#25, c_customer_sk#21, c_first_name#22, c_last_name#23] +Arguments: [c_last_name#23, c_first_name#22, substr(s_city, 1, 30)#26, ss_ticket_number#5, amt#24, profit#25, s_city#15], [c_last_name#23, c_first_name#22, substr(s_city#15, 1, 30) AS substr(s_city, 1, 30)#26, ss_ticket_number#5, amt#24, profit#25, s_city#15] -(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] +(29) CometTakeOrderedAndProject +Input [7]: [c_last_name#23, c_first_name#22, substr(s_city, 1, 30)#26, ss_ticket_number#5, amt#24, profit#25, s_city#15] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#23 ASC NULLS FIRST,c_first_name#22 ASC NULLS FIRST,substr(s_city#15, 1, 30) ASC NULLS FIRST,profit#25 ASC NULLS FIRST], output=[c_last_name#23,c_first_name#22,substr(s_city, 1, 30)#26,ss_ticket_number#5,amt#24,profit#25]), [c_last_name#23, c_first_name#22, substr(s_city, 1, 30)#26, ss_ticket_number#5, amt#24, profit#25], 100, [c_last_name#23 ASC NULLS FIRST, c_first_name#22 ASC NULLS FIRST, substr(s_city#15, 1, 30) ASC NULLS FIRST, profit#25 ASC NULLS FIRST], [c_last_name#23, c_first_name#22, substr(s_city, 1, 30)#26, ss_ticket_number#5, amt#24, profit#25] -(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] +(30) ColumnarToRow [codegen id : 1] +Input [6]: [c_last_name#23, c_first_name#22, substr(s_city, 1, 30)#26, ss_ticket_number#5, amt#24, profit#25] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (36) -+- * ColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan parquet spark_catalog.default.date_dim (32) +BroadcastExchange (35) ++- * ColumnarToRow (34) + +- CometProject (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.date_dim (31) -(32) Scan parquet spark_catalog.default.date_dim +(31) 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 -(33) CometFilter +(32) 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)) -(34) CometProject +(33) CometProject Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(35) ColumnarToRow [codegen id : 1] +(34) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(36) BroadcastExchange +(35) BroadcastExchange Input [1]: [d_date_sk#10] -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] 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 57598ec35..6e6401470 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,46 +1,40 @@ -TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, 30),ss_ticket_number,amt] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum,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_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city,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_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_store_sk,s_city] - CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [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,d_date_sk] - CometFilter [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] - 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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [s_store_sk,s_city] #4 - CometProject [s_store_sk,s_city] - CometFilter [s_store_sk,s_number_employees,s_city] - CometScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] - CometBroadcastExchange [hd_demo_sk] #5 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [c_last_name,c_first_name,substr(s_city, 1, 30),ss_ticket_number,amt,profit,s_city] + CometProject [c_last_name,c_first_name,substr(s_city, 1, 30),ss_ticket_number,amt,profit,s_city] + CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,s_city,amt,profit,c_customer_sk,c_first_name,c_last_name] + CometHashAggregate [ss_ticket_number,ss_customer_sk,s_city,amt,profit,ss_addr_sk,sum,sum,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] + CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city] #1 + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum,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_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city,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_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_store_sk,s_city] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + CometBroadcastHashJoin [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,d_date_sk] + CometFilter [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] + 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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometBroadcastExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometBroadcastExchange [s_store_sk,s_city] #4 + CometProject [s_store_sk,s_city] + CometFilter [s_store_sk,s_number_employees,s_city] + CometScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] + CometBroadcastExchange [hd_demo_sk] #5 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #6 + CometFilter [c_customer_sk,c_first_name,c_last_name] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] 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 07bfd877c..19d7ccc99 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,48 +1,46 @@ == Physical Plan == -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) +* ColumnarToRow (42) ++- CometTakeOrderedAndProject (41) + +- CometHashAggregate (40) + +- CometColumnarExchange (39) + +- CometHashAggregate (38) + +- CometProject (37) + +- CometBroadcastHashJoin (36) + :- 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 (35) + +- CometHashAggregate (34) + +- CometColumnarExchange (33) + +- CometHashAggregate (32) + +- CometBroadcastHashJoin (31) + :- CometProject (16) + : +- CometFilter (15) + : +- CometScan parquet spark_catalog.default.customer_address (14) + +- CometBroadcastExchange (30) + +- CometProject (29) + +- CometFilter (28) + +- CometHashAggregate (27) + +- CometColumnarExchange (26) + +- CometHashAggregate (25) + +- CometProject (24) + +- CometBroadcastHashJoin (23) + :- CometFilter (18) + : +- CometScan parquet spark_catalog.default.customer_address (17) + +- CometBroadcastExchange (22) + +- CometProject (21) + +- CometFilter (20) + +- CometScan parquet spark_catalog.default.customer (19) (1) Scan parquet spark_catalog.default.store_sales @@ -109,186 +107,168 @@ Arguments: [ss_store_sk#1], [s_store_sk#8], Inner, BuildRight 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] -(14) ColumnarToRow [codegen id : 5] -Input [3]: [ss_net_profit#2, s_store_name#9, s_zip#10] - -(15) Scan parquet spark_catalog.default.customer_address +(14) 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 -(16) CometFilter +(15) 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))) -(17) CometProject +(16) CometProject Input [1]: [ca_zip#11] Arguments: [ca_zip#12], [substr(ca_zip#11, 1, 5) AS ca_zip#12] -(18) ColumnarToRow [codegen id : 3] -Input [1]: [ca_zip#12] - -(19) Scan parquet spark_catalog.default.customer_address +(17) 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 -(20) CometFilter +(18) CometFilter Input [2]: [ca_address_sk#13, ca_zip#14] Condition : isnotnull(ca_address_sk#13) -(21) Scan parquet spark_catalog.default.customer +(19) 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 -(22) CometFilter +(20) 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)) -(23) CometProject +(21) CometProject Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] Arguments: [c_current_addr_sk#15], [c_current_addr_sk#15] -(24) CometBroadcastExchange +(22) CometBroadcastExchange Input [1]: [c_current_addr_sk#15] Arguments: [c_current_addr_sk#15] -(25) CometBroadcastHashJoin +(23) 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 -(26) CometProject +(24) CometProject Input [3]: [ca_address_sk#13, ca_zip#14, c_current_addr_sk#15] Arguments: [ca_zip#14], [ca_zip#14] -(27) CometHashAggregate +(25) CometHashAggregate Input [1]: [ca_zip#14] Keys [1]: [ca_zip#14] Functions [1]: [partial_count(1)] -(28) ColumnarToRow [codegen id : 1] -Input [2]: [ca_zip#14, count#17] - -(29) Exchange +(26) CometColumnarExchange Input [2]: [ca_zip#14, count#17] -Arguments: hashpartitioning(ca_zip#14, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: hashpartitioning(ca_zip#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(30) HashAggregate [codegen id : 2] +(27) CometHashAggregate Input [2]: [ca_zip#14, count#17] Keys [1]: [ca_zip#14] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#18] -Results [2]: [substr(ca_zip#14, 1, 5) AS ca_zip#19, count(1)#18 AS cnt#20] -(31) Filter [codegen id : 2] -Input [2]: [ca_zip#19, cnt#20] -Condition : (cnt#20 > 10) +(28) CometFilter +Input [2]: [ca_zip#18, cnt#19] +Condition : (cnt#19 > 10) -(32) Project [codegen id : 2] -Output [1]: [ca_zip#19] -Input [2]: [ca_zip#19, cnt#20] +(29) CometProject +Input [2]: [ca_zip#18, cnt#19] +Arguments: [ca_zip#18], [ca_zip#18] -(33) BroadcastExchange -Input [1]: [ca_zip#19] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [plan_id=2] +(30) CometBroadcastExchange +Input [1]: [ca_zip#18] +Arguments: [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 +(31) CometBroadcastHashJoin +Left output [1]: [ca_zip#12] +Right output [1]: [ca_zip#18] +Arguments: [coalesce(ca_zip#12, ), isnull(ca_zip#12)], [coalesce(ca_zip#18, ), isnull(ca_zip#18)], LeftSemi, BuildRight -(35) HashAggregate [codegen id : 3] +(32) CometHashAggregate Input [1]: [ca_zip#12] Keys [1]: [ca_zip#12] Functions: [] -Aggregate Attributes: [] -Results [1]: [ca_zip#12] -(36) Exchange +(33) CometColumnarExchange Input [1]: [ca_zip#12] -Arguments: hashpartitioning(ca_zip#12, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(ca_zip#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(37) HashAggregate [codegen id : 4] +(34) CometHashAggregate Input [1]: [ca_zip#12] Keys [1]: [ca_zip#12] Functions: [] -Aggregate Attributes: [] -Results [1]: [ca_zip#12] -(38) BroadcastExchange +(35) CometBroadcastExchange Input [1]: [ca_zip#12] -Arguments: HashedRelationBroadcastMode(List(substr(input[0, string, true], 1, 2)),false), [plan_id=4] +Arguments: [ca_zip#12] -(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 +(36) CometBroadcastHashJoin +Left output [3]: [ss_net_profit#2, s_store_name#9, s_zip#10] +Right output [1]: [ca_zip#12] +Arguments: [substr(s_zip#10, 1, 2)], [substr(ca_zip#12, 1, 2)], Inner, BuildRight -(40) Project [codegen id : 5] -Output [2]: [ss_net_profit#2, s_store_name#9] +(37) CometProject Input [4]: [ss_net_profit#2, s_store_name#9, s_zip#10, ca_zip#12] +Arguments: [ss_net_profit#2, s_store_name#9], [ss_net_profit#2, s_store_name#9] -(41) HashAggregate [codegen id : 5] +(38) CometHashAggregate 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#21] -Results [2]: [s_store_name#9, sum#22] -(42) Exchange -Input [2]: [s_store_name#9, sum#22] -Arguments: hashpartitioning(s_store_name#9, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(39) CometColumnarExchange +Input [2]: [s_store_name#9, sum#20] +Arguments: hashpartitioning(s_store_name#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(43) HashAggregate [codegen id : 6] -Input [2]: [s_store_name#9, sum#22] +(40) CometHashAggregate +Input [2]: [s_store_name#9, sum#20] Keys [1]: [s_store_name#9] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -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] -(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] +(41) CometTakeOrderedAndProject +Input [2]: [s_store_name#9, sum(ss_net_profit)#21] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#9 ASC NULLS FIRST], output=[s_store_name#9,sum(ss_net_profit)#21]), [s_store_name#9, sum(ss_net_profit)#21], 100, [s_store_name#9 ASC NULLS FIRST], [s_store_name#9, sum(ss_net_profit)#21] + +(42) ColumnarToRow [codegen id : 1] +Input [2]: [s_store_name#9, sum(ss_net_profit)#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (49) -+- * ColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan parquet spark_catalog.default.date_dim (45) +BroadcastExchange (47) ++- * ColumnarToRow (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) -(45) Scan parquet spark_catalog.default.date_dim +(43) 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 -(46) CometFilter +(44) 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)) -(47) CometProject +(45) CometProject Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Arguments: [d_date_sk#5], [d_date_sk#5] -(48) ColumnarToRow [codegen id : 1] +(46) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(49) BroadcastExchange +(47) BroadcastExchange Input [1]: [d_date_sk#5] -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/q8/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt index 637f3b7ba..6e1a3d332 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,66 +1,52 @@ -TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] - WholeStageCodegen (6) - HashAggregate [s_store_name,sum] [sum(UnscaledValue(ss_net_profit)),sum(ss_net_profit),sum] - InputAdapter - Exchange [s_store_name] #1 - WholeStageCodegen (5) - HashAggregate [s_store_name,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_store_name] - BroadcastHashJoin [s_zip,ca_zip] - ColumnarToRow - InputAdapter - CometProject [ss_net_profit,s_store_name,s_zip] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,s_store_sk,s_store_name,s_zip] - CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #4 - CometFilter [s_store_sk,s_store_name,s_zip] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - HashAggregate [ca_zip] - InputAdapter - Exchange [ca_zip] #6 - WholeStageCodegen (3) - HashAggregate [ca_zip] - BroadcastHashJoin [ca_zip,ca_zip] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [s_store_name,sum(ss_net_profit)] + CometHashAggregate [s_store_name,sum(ss_net_profit),sum,sum(UnscaledValue(ss_net_profit))] + CometColumnarExchange [s_store_name] #1 + CometHashAggregate [s_store_name,sum,ss_net_profit] + CometProject [ss_net_profit,s_store_name] + CometBroadcastHashJoin [ss_net_profit,s_store_name,s_zip,ca_zip] + CometProject [ss_net_profit,s_store_name,s_zip] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,s_store_sk,s_store_name,s_zip] + CometProject [ss_store_sk,ss_net_profit] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [ca_zip] [ca_zip] - CometFilter [ca_zip] - CometScan parquet spark_catalog.default.customer_address [ca_zip] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - Project [ca_zip] - Filter [cnt] - HashAggregate [ca_zip,count] [count(1),ca_zip,cnt,count] - InputAdapter - Exchange [ca_zip] #8 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ca_zip,count] - CometProject [ca_zip] - CometBroadcastHashJoin [ca_address_sk,ca_zip,c_current_addr_sk] - CometFilter [ca_address_sk,ca_zip] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange [c_current_addr_sk] #9 - CometProject [c_current_addr_sk] - CometFilter [c_current_addr_sk,c_preferred_cust_flag] - CometScan parquet spark_catalog.default.customer [c_current_addr_sk,c_preferred_cust_flag] + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #4 + CometFilter [s_store_sk,s_store_name,s_zip] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] + CometBroadcastExchange [ca_zip] #5 + CometHashAggregate [ca_zip] + CometColumnarExchange [ca_zip] #6 + CometHashAggregate [ca_zip] + CometBroadcastHashJoin [ca_zip,ca_zip] + CometProject [ca_zip] [ca_zip] + CometFilter [ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_zip] + CometBroadcastExchange [ca_zip] #7 + CometProject [ca_zip] + CometFilter [ca_zip,cnt] + CometHashAggregate [ca_zip,cnt,ca_zip,count,count(1)] + CometColumnarExchange [ca_zip] #8 + CometHashAggregate [ca_zip,count] + CometProject [ca_zip] + CometBroadcastHashJoin [ca_address_sk,ca_zip,c_current_addr_sk] + CometFilter [ca_address_sk,ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] + CometBroadcastExchange [c_current_addr_sk] #9 + CometProject [c_current_addr_sk] + CometFilter [c_current_addr_sk,c_preferred_cust_flag] + 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/q80/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt index db2d015db..a66d4f66d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt @@ -1,111 +1,108 @@ == Physical Plan == -TakeOrderedAndProject (107) -+- * HashAggregate (106) - +- Exchange (105) - +- * HashAggregate (104) - +- * Expand (103) - +- Union (102) - :- * HashAggregate (39) - : +- Exchange (38) - : +- * HashAggregate (37) - : +- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * Project (29) - : : +- * BroadcastHashJoin Inner BuildRight (28) - : : :- * Project (22) - : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : :- * Project (16) - : : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : : :- * Project (13) - : : : : : +- * SortMergeJoin LeftOuter (12) - : : : : : :- * Sort (5) - : : : : : : +- Exchange (4) - : : : : : : +- * ColumnarToRow (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : +- * Sort (11) - : : : : : +- Exchange (10) - : : : : : +- * ColumnarToRow (9) - : : : : : +- CometProject (8) - : : : : : +- CometFilter (7) - : : : : : +- CometScan parquet spark_catalog.default.store_returns (6) - : : : : +- ReusedExchange (14) - : : : +- BroadcastExchange (20) - : : : +- * ColumnarToRow (19) - : : : +- CometFilter (18) - : : : +- CometScan parquet spark_catalog.default.store (17) - : : +- BroadcastExchange (27) - : : +- * ColumnarToRow (26) - : : +- CometProject (25) - : : +- CometFilter (24) - : : +- CometScan parquet spark_catalog.default.item (23) - : +- BroadcastExchange (34) - : +- * ColumnarToRow (33) - : +- CometProject (32) - : +- CometFilter (31) - : +- CometScan parquet spark_catalog.default.promotion (30) - :- * HashAggregate (70) - : +- Exchange (69) - : +- * HashAggregate (68) - : +- * Project (67) - : +- * BroadcastHashJoin Inner BuildRight (66) - : :- * Project (64) - : : +- * BroadcastHashJoin Inner BuildRight (63) - : : :- * Project (61) - : : : +- * BroadcastHashJoin Inner BuildRight (60) - : : : :- * Project (55) - : : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : : :- * Project (52) - : : : : : +- * SortMergeJoin LeftOuter (51) - : : : : : :- * Sort (44) - : : : : : : +- Exchange (43) - : : : : : : +- * ColumnarToRow (42) - : : : : : : +- CometFilter (41) - : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (40) - : : : : : +- * Sort (50) - : : : : : +- Exchange (49) - : : : : : +- * ColumnarToRow (48) - : : : : : +- CometProject (47) - : : : : : +- CometFilter (46) - : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (45) - : : : : +- ReusedExchange (53) - : : : +- BroadcastExchange (59) - : : : +- * ColumnarToRow (58) - : : : +- CometFilter (57) - : : : +- CometScan parquet spark_catalog.default.catalog_page (56) - : : +- ReusedExchange (62) - : +- ReusedExchange (65) - +- * HashAggregate (101) - +- Exchange (100) - +- * HashAggregate (99) - +- * Project (98) - +- * BroadcastHashJoin Inner BuildRight (97) - :- * Project (95) - : +- * BroadcastHashJoin Inner BuildRight (94) - : :- * Project (92) - : : +- * BroadcastHashJoin Inner BuildRight (91) - : : :- * Project (86) - : : : +- * BroadcastHashJoin Inner BuildRight (85) - : : : :- * Project (83) - : : : : +- * SortMergeJoin LeftOuter (82) - : : : : :- * Sort (75) - : : : : : +- Exchange (74) - : : : : : +- * ColumnarToRow (73) - : : : : : +- CometFilter (72) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (71) - : : : : +- * Sort (81) - : : : : +- Exchange (80) - : : : : +- * ColumnarToRow (79) - : : : : +- CometProject (78) - : : : : +- CometFilter (77) - : : : : +- CometScan parquet spark_catalog.default.web_returns (76) - : : : +- ReusedExchange (84) - : : +- BroadcastExchange (90) - : : +- * ColumnarToRow (89) - : : +- CometFilter (88) - : : +- CometScan parquet spark_catalog.default.web_site (87) - : +- ReusedExchange (93) - +- ReusedExchange (96) +TakeOrderedAndProject (104) ++- * HashAggregate (103) + +- * ColumnarToRow (102) + +- CometColumnarExchange (101) + +- RowToColumnar (100) + +- * HashAggregate (99) + +- * Expand (98) + +- Union (97) + :- * HashAggregate (38) + : +- * ColumnarToRow (37) + : +- CometColumnarExchange (36) + : +- CometHashAggregate (35) + : +- CometProject (34) + : +- CometBroadcastHashJoin (33) + : :- CometProject (28) + : : +- CometBroadcastHashJoin (27) + : : :- CometProject (22) + : : : +- CometBroadcastHashJoin (21) + : : : :- CometProject (17) + : : : : +- CometBroadcastHashJoin (16) + : : : : :- CometProject (11) + : : : : : +- CometSortMergeJoin (10) + : : : : : :- CometSort (4) + : : : : : : +- CometColumnarExchange (3) + : : : : : : +- CometFilter (2) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : +- CometSort (9) + : : : : : +- CometColumnarExchange (8) + : : : : : +- CometProject (7) + : : : : : +- CometFilter (6) + : : : : : +- CometScan parquet spark_catalog.default.store_returns (5) + : : : : +- CometBroadcastExchange (15) + : : : : +- CometProject (14) + : : : : +- CometFilter (13) + : : : : +- CometScan parquet spark_catalog.default.date_dim (12) + : : : +- CometBroadcastExchange (20) + : : : +- CometFilter (19) + : : : +- CometScan parquet spark_catalog.default.store (18) + : : +- CometBroadcastExchange (26) + : : +- CometProject (25) + : : +- CometFilter (24) + : : +- CometScan parquet spark_catalog.default.item (23) + : +- CometBroadcastExchange (32) + : +- CometProject (31) + : +- CometFilter (30) + : +- CometScan parquet spark_catalog.default.promotion (29) + :- * HashAggregate (67) + : +- * ColumnarToRow (66) + : +- CometColumnarExchange (65) + : +- CometHashAggregate (64) + : +- CometProject (63) + : +- CometBroadcastHashJoin (62) + : :- CometProject (60) + : : +- CometBroadcastHashJoin (59) + : : :- CometProject (57) + : : : +- CometBroadcastHashJoin (56) + : : : :- CometProject (52) + : : : : +- CometBroadcastHashJoin (51) + : : : : :- CometProject (49) + : : : : : +- CometSortMergeJoin (48) + : : : : : :- CometSort (42) + : : : : : : +- CometColumnarExchange (41) + : : : : : : +- CometFilter (40) + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (39) + : : : : : +- CometSort (47) + : : : : : +- CometColumnarExchange (46) + : : : : : +- CometProject (45) + : : : : : +- CometFilter (44) + : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (43) + : : : : +- ReusedExchange (50) + : : : +- CometBroadcastExchange (55) + : : : +- CometFilter (54) + : : : +- CometScan parquet spark_catalog.default.catalog_page (53) + : : +- ReusedExchange (58) + : +- ReusedExchange (61) + +- * HashAggregate (96) + +- * ColumnarToRow (95) + +- CometColumnarExchange (94) + +- CometHashAggregate (93) + +- CometProject (92) + +- CometBroadcastHashJoin (91) + :- CometProject (89) + : +- CometBroadcastHashJoin (88) + : :- CometProject (86) + : : +- CometBroadcastHashJoin (85) + : : :- CometProject (81) + : : : +- CometBroadcastHashJoin (80) + : : : :- CometProject (78) + : : : : +- CometSortMergeJoin (77) + : : : : :- CometSort (71) + : : : : : +- CometColumnarExchange (70) + : : : : : +- CometFilter (69) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (68) + : : : : +- CometSort (76) + : : : : +- CometColumnarExchange (75) + : : : : +- CometProject (74) + : : : : +- CometFilter (73) + : : : : +- CometScan parquet spark_catalog.default.web_returns (72) + : : : +- ReusedExchange (79) + : : +- CometBroadcastExchange (84) + : : +- CometFilter (83) + : : +- CometScan parquet spark_catalog.default.web_site (82) + : +- ReusedExchange (87) + +- ReusedExchange (90) (1) Scan parquet spark_catalog.default.store_sales @@ -120,526 +117,503 @@ ReadSchema: struct -(7) CometFilter +(6) CometFilter Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] Condition : (isnotnull(sr_item_sk#9) AND isnotnull(sr_ticket_number#10)) -(8) CometProject +(7) CometProject Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -(9) ColumnarToRow [codegen id : 3] -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] - -(10) Exchange +(8) CometColumnarExchange Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(11) Sort [codegen id : 4] +(9) CometSort Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST], false, 0 +Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST] -(12) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_item_sk#1, ss_ticket_number#4] -Right keys [2]: [sr_item_sk#9, sr_ticket_number#10] -Join type: LeftOuter -Join condition: None +(10) CometSortMergeJoin +Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] +Right output [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] +Arguments: [ss_item_sk#1, ss_ticket_number#4], [sr_item_sk#9, sr_ticket_number#10], LeftOuter -(13) Project [codegen id : 9] -Output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] +(11) CometProject Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] +Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] -(14) ReusedExchange [Reuses operator id: 112] -Output [1]: [d_date_sk#14] +(12) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#15] +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-22), IsNotNull(d_date_sk)] +ReadSchema: struct -(15) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None +(13) CometFilter +Input [2]: [d_date_sk#14, d_date#15] +Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#15 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) -(16) Project [codegen id : 9] -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] +(14) CometProject +Input [2]: [d_date_sk#14, d_date#15] +Arguments: [d_date_sk#14], [d_date_sk#14] + +(15) CometBroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: [d_date_sk#14] + +(16) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] +Right output [1]: [d_date_sk#14] +Arguments: [ss_sold_date_sk#7], [d_date_sk#14], Inner, BuildRight + +(17) CometProject Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#14] +Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] -(17) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#15, s_store_id#16] +(18) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#16, s_store_id#17] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(18) CometFilter -Input [2]: [s_store_sk#15, s_store_id#16] -Condition : isnotnull(s_store_sk#15) - -(19) ColumnarToRow [codegen id : 6] -Input [2]: [s_store_sk#15, s_store_id#16] +(19) CometFilter +Input [2]: [s_store_sk#16, s_store_id#17] +Condition : isnotnull(s_store_sk#16) -(20) BroadcastExchange -Input [2]: [s_store_sk#15, s_store_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(20) CometBroadcastExchange +Input [2]: [s_store_sk#16, s_store_id#17] +Arguments: [s_store_sk#16, s_store_id#17] -(21) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#15] -Join type: Inner -Join condition: None +(21) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] +Right output [2]: [s_store_sk#16, s_store_id#17] +Arguments: [ss_store_sk#2], [s_store_sk#16], Inner, BuildRight -(22) Project [codegen id : 9] -Output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#15, s_store_id#16] +(22) CometProject +Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#16, s_store_id#17] +Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17], [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] (23) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#17, i_current_price#18] +Output [2]: [i_item_sk#18, i_current_price#19] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), GreaterThan(i_current_price,50.00), IsNotNull(i_item_sk)] ReadSchema: struct (24) CometFilter -Input [2]: [i_item_sk#17, i_current_price#18] -Condition : ((isnotnull(i_current_price#18) AND (i_current_price#18 > 50.00)) AND isnotnull(i_item_sk#17)) +Input [2]: [i_item_sk#18, i_current_price#19] +Condition : ((isnotnull(i_current_price#19) AND (i_current_price#19 > 50.00)) AND isnotnull(i_item_sk#18)) (25) CometProject -Input [2]: [i_item_sk#17, i_current_price#18] -Arguments: [i_item_sk#17], [i_item_sk#17] - -(26) ColumnarToRow [codegen id : 7] -Input [1]: [i_item_sk#17] +Input [2]: [i_item_sk#18, i_current_price#19] +Arguments: [i_item_sk#18], [i_item_sk#18] -(27) BroadcastExchange -Input [1]: [i_item_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(26) CometBroadcastExchange +Input [1]: [i_item_sk#18] +Arguments: [i_item_sk#18] -(28) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#17] -Join type: Inner -Join condition: None +(27) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] +Right output [1]: [i_item_sk#18] +Arguments: [ss_item_sk#1], [i_item_sk#18], Inner, BuildRight -(29) Project [codegen id : 9] -Output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16, i_item_sk#17] +(28) CometProject +Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17, i_item_sk#18] +Arguments: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17], [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] -(30) Scan parquet spark_catalog.default.promotion -Output [2]: [p_promo_sk#19, p_channel_tv#20] +(29) Scan parquet spark_catalog.default.promotion +Output [2]: [p_promo_sk#20, p_channel_tv#21] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_channel_tv), EqualTo(p_channel_tv,N), IsNotNull(p_promo_sk)] ReadSchema: struct -(31) CometFilter -Input [2]: [p_promo_sk#19, p_channel_tv#20] -Condition : ((isnotnull(p_channel_tv#20) AND (p_channel_tv#20 = N)) AND isnotnull(p_promo_sk#19)) +(30) CometFilter +Input [2]: [p_promo_sk#20, p_channel_tv#21] +Condition : ((isnotnull(p_channel_tv#21) AND (p_channel_tv#21 = N)) AND isnotnull(p_promo_sk#20)) -(32) CometProject -Input [2]: [p_promo_sk#19, p_channel_tv#20] -Arguments: [p_promo_sk#19], [p_promo_sk#19] +(31) CometProject +Input [2]: [p_promo_sk#20, p_channel_tv#21] +Arguments: [p_promo_sk#20], [p_promo_sk#20] -(33) ColumnarToRow [codegen id : 8] -Input [1]: [p_promo_sk#19] +(32) CometBroadcastExchange +Input [1]: [p_promo_sk#20] +Arguments: [p_promo_sk#20] -(34) BroadcastExchange -Input [1]: [p_promo_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(33) CometBroadcastHashJoin +Left output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] +Right output [1]: [p_promo_sk#20] +Arguments: [ss_promo_sk#3], [p_promo_sk#20], Inner, BuildRight -(35) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_promo_sk#3] -Right keys [1]: [p_promo_sk#19] -Join type: Inner -Join condition: None +(34) CometProject +Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17, p_promo_sk#20] +Arguments: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17], [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] -(36) Project [codegen id : 9] -Output [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] -Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16, p_promo_sk#19] - -(37) HashAggregate [codegen id : 9] -Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] -Keys [1]: [s_store_id#16] +(35) CometHashAggregate +Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] +Keys [1]: [s_store_id#17] Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] -Aggregate Attributes [5]: [sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -Results [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] -(38) Exchange -Input [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] -Arguments: hashpartitioning(s_store_id#16, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(36) CometColumnarExchange +Input [6]: [s_store_id#17, sum#22, sum#23, isEmpty#24, sum#25, isEmpty#26] +Arguments: hashpartitioning(s_store_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(37) ColumnarToRow [codegen id : 1] +Input [6]: [s_store_id#17, sum#22, sum#23, isEmpty#24, sum#25, isEmpty#26] -(39) HashAggregate [codegen id : 10] -Input [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] -Keys [1]: [s_store_id#16] +(38) HashAggregate [codegen id : 1] +Input [6]: [s_store_id#17, sum#22, sum#23, isEmpty#24, sum#25, isEmpty#26] +Keys [1]: [s_store_id#17] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#5))#31, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#32, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#33] -Results [5]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#31,17,2) AS sales#34, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#32 AS returns#35, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#33 AS profit#36, store channel AS channel#37, concat(store, s_store_id#16) AS id#38] +Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#5))#27, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#28, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#29] +Results [5]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#27,17,2) AS sales#30, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#28 AS returns#31, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#29 AS profit#32, store channel AS channel#33, concat(store, s_store_id#17) AS id#34] -(40) Scan parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +(39) Scan parquet spark_catalog.default.catalog_sales +Output [7]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_order_number#38, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41] 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#41), dynamicpruningexpression(cs_sold_date_sk#41 IN dynamicpruning#42)] PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] ReadSchema: struct -(41) CometFilter -Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Condition : ((isnotnull(cs_catalog_page_sk#39) AND isnotnull(cs_item_sk#40)) AND isnotnull(cs_promo_sk#41)) +(40) CometFilter +Input [7]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_order_number#38, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41] +Condition : ((isnotnull(cs_catalog_page_sk#35) AND isnotnull(cs_item_sk#36)) AND isnotnull(cs_promo_sk#37)) -(42) ColumnarToRow [codegen id : 11] -Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +(41) CometColumnarExchange +Input [7]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_order_number#38, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41] +Arguments: hashpartitioning(cs_item_sk#36, cs_order_number#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(43) Exchange -Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Arguments: hashpartitioning(cs_item_sk#40, cs_order_number#42, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(42) CometSort +Input [7]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_order_number#38, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41] +Arguments: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_order_number#38, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41], [cs_item_sk#36 ASC NULLS FIRST, cs_order_number#38 ASC NULLS FIRST] -(44) Sort [codegen id : 12] -Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Arguments: [cs_item_sk#40 ASC NULLS FIRST, cs_order_number#42 ASC NULLS FIRST], false, 0 - -(45) Scan parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] +(43) Scan parquet spark_catalog.default.catalog_returns +Output [5]: [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46, cr_returned_date_sk#47] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(46) CometFilter -Input [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] -Condition : (isnotnull(cr_item_sk#47) AND isnotnull(cr_order_number#48)) - -(47) CometProject -Input [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] -Arguments: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50], [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] +(44) CometFilter +Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46, cr_returned_date_sk#47] +Condition : (isnotnull(cr_item_sk#43) AND isnotnull(cr_order_number#44)) -(48) ColumnarToRow [codegen id : 13] -Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] +(45) CometProject +Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46, cr_returned_date_sk#47] +Arguments: [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46], [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46] -(49) Exchange -Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -Arguments: hashpartitioning(cr_item_sk#47, cr_order_number#48, 5), ENSURE_REQUIREMENTS, [plan_id=8] +(46) CometColumnarExchange +Input [4]: [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46] +Arguments: hashpartitioning(cr_item_sk#43, cr_order_number#44, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(50) Sort [codegen id : 14] -Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -Arguments: [cr_item_sk#47 ASC NULLS FIRST, cr_order_number#48 ASC NULLS FIRST], false, 0 +(47) CometSort +Input [4]: [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46] +Arguments: [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46], [cr_item_sk#43 ASC NULLS FIRST, cr_order_number#44 ASC NULLS FIRST] -(51) SortMergeJoin [codegen id : 19] -Left keys [2]: [cs_item_sk#40, cs_order_number#42] -Right keys [2]: [cr_item_sk#47, cr_order_number#48] -Join type: LeftOuter -Join condition: None +(48) CometSortMergeJoin +Left output [7]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_order_number#38, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41] +Right output [4]: [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46] +Arguments: [cs_item_sk#36, cs_order_number#38], [cr_item_sk#43, cr_order_number#44], LeftOuter -(52) Project [codegen id : 19] -Output [8]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50] -Input [11]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] +(49) CometProject +Input [11]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_order_number#38, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41, cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46] +Arguments: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41, cr_return_amount#45, cr_net_loss#46], [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41, cr_return_amount#45, cr_net_loss#46] -(53) ReusedExchange [Reuses operator id: 112] -Output [1]: [d_date_sk#52] +(50) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#48] -(54) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_sold_date_sk#45] -Right keys [1]: [d_date_sk#52] -Join type: Inner -Join condition: None +(51) CometBroadcastHashJoin +Left output [8]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41, cr_return_amount#45, cr_net_loss#46] +Right output [1]: [d_date_sk#48] +Arguments: [cs_sold_date_sk#41], [d_date_sk#48], Inner, BuildRight -(55) Project [codegen id : 19] -Output [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50] -Input [9]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50, d_date_sk#52] +(52) CometProject +Input [9]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41, cr_return_amount#45, cr_net_loss#46, d_date_sk#48] +Arguments: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46], [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46] -(56) Scan parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +(53) Scan parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#49, cp_catalog_page_id#50] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(57) CometFilter -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Condition : isnotnull(cp_catalog_page_sk#53) - -(58) ColumnarToRow [codegen id : 16] -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] - -(59) BroadcastExchange -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -(60) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_catalog_page_sk#39] -Right keys [1]: [cp_catalog_page_sk#53] -Join type: Inner -Join condition: None - -(61) Project [codegen id : 19] -Output [7]: [cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] -Input [9]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_sk#53, cp_catalog_page_id#54] - -(62) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#55] - -(63) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_item_sk#40] -Right keys [1]: [i_item_sk#55] -Join type: Inner -Join condition: None - -(64) Project [codegen id : 19] -Output [6]: [cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] -Input [8]: [cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54, i_item_sk#55] - -(65) ReusedExchange [Reuses operator id: 34] -Output [1]: [p_promo_sk#56] - -(66) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_promo_sk#41] -Right keys [1]: [p_promo_sk#56] -Join type: Inner -Join condition: None - -(67) Project [codegen id : 19] -Output [5]: [cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] -Input [7]: [cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54, p_promo_sk#56] - -(68) HashAggregate [codegen id : 19] -Input [5]: [cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] -Keys [1]: [cp_catalog_page_id#54] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#43)), partial_sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))] -Aggregate Attributes [5]: [sum#57, sum#58, isEmpty#59, sum#60, isEmpty#61] -Results [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] - -(69) Exchange -Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] -Arguments: hashpartitioning(cp_catalog_page_id#54, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(70) HashAggregate [codegen id : 20] -Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] -Keys [1]: [cp_catalog_page_id#54] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#43)), sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00)), sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#43))#67, sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00))#68, sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))#69] -Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#43))#67,17,2) AS sales#70, sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00))#68 AS returns#71, sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))#69 AS profit#72, catalog channel AS channel#73, concat(catalog_page, cp_catalog_page_id#54) AS id#74] - -(71) Scan parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] +(54) CometFilter +Input [2]: [cp_catalog_page_sk#49, cp_catalog_page_id#50] +Condition : isnotnull(cp_catalog_page_sk#49) + +(55) CometBroadcastExchange +Input [2]: [cp_catalog_page_sk#49, cp_catalog_page_id#50] +Arguments: [cp_catalog_page_sk#49, cp_catalog_page_id#50] + +(56) CometBroadcastHashJoin +Left output [7]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46] +Right output [2]: [cp_catalog_page_sk#49, cp_catalog_page_id#50] +Arguments: [cs_catalog_page_sk#35], [cp_catalog_page_sk#49], Inner, BuildRight + +(57) CometProject +Input [9]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_sk#49, cp_catalog_page_id#50] +Arguments: [cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50], [cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50] + +(58) ReusedExchange [Reuses operator id: 26] +Output [1]: [i_item_sk#51] + +(59) CometBroadcastHashJoin +Left output [7]: [cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50] +Right output [1]: [i_item_sk#51] +Arguments: [cs_item_sk#36], [i_item_sk#51], Inner, BuildRight + +(60) CometProject +Input [8]: [cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50, i_item_sk#51] +Arguments: [cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50], [cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50] + +(61) ReusedExchange [Reuses operator id: 32] +Output [1]: [p_promo_sk#52] + +(62) CometBroadcastHashJoin +Left output [6]: [cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50] +Right output [1]: [p_promo_sk#52] +Arguments: [cs_promo_sk#37], [p_promo_sk#52], Inner, BuildRight + +(63) CometProject +Input [7]: [cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50, p_promo_sk#52] +Arguments: [cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50], [cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50] + +(64) CometHashAggregate +Input [5]: [cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50] +Keys [1]: [cp_catalog_page_id#50] +Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#39)), partial_sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#40 - coalesce(cast(cr_net_loss#46 as decimal(12,2)), 0.00)))] + +(65) CometColumnarExchange +Input [6]: [cp_catalog_page_id#50, sum#53, sum#54, isEmpty#55, sum#56, isEmpty#57] +Arguments: hashpartitioning(cp_catalog_page_id#50, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(66) ColumnarToRow [codegen id : 2] +Input [6]: [cp_catalog_page_id#50, sum#53, sum#54, isEmpty#55, sum#56, isEmpty#57] + +(67) HashAggregate [codegen id : 2] +Input [6]: [cp_catalog_page_id#50, sum#53, sum#54, isEmpty#55, sum#56, isEmpty#57] +Keys [1]: [cp_catalog_page_id#50] +Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#39)), sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00)), sum((cs_net_profit#40 - coalesce(cast(cr_net_loss#46 as decimal(12,2)), 0.00)))] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#39))#58, sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00))#59, sum((cs_net_profit#40 - coalesce(cast(cr_net_loss#46 as decimal(12,2)), 0.00)))#60] +Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#39))#58,17,2) AS sales#61, sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00))#59 AS returns#62, sum((cs_net_profit#40 - coalesce(cast(cr_net_loss#46 as decimal(12,2)), 0.00)))#60 AS profit#63, catalog channel AS channel#64, concat(catalog_page, cp_catalog_page_id#50) AS id#65] + +(68) Scan parquet spark_catalog.default.web_sales +Output [7]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_order_number#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] 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#72), dynamicpruningexpression(ws_sold_date_sk#72 IN dynamicpruning#73)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct -(72) CometFilter -Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Condition : ((isnotnull(ws_web_site_sk#76) AND isnotnull(ws_item_sk#75)) AND isnotnull(ws_promo_sk#77)) +(69) CometFilter +Input [7]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_order_number#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Condition : ((isnotnull(ws_web_site_sk#67) AND isnotnull(ws_item_sk#66)) AND isnotnull(ws_promo_sk#68)) -(73) ColumnarToRow [codegen id : 21] -Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] +(70) CometColumnarExchange +Input [7]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_order_number#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Arguments: hashpartitioning(ws_item_sk#66, ws_order_number#69, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(74) Exchange -Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Arguments: hashpartitioning(ws_item_sk#75, ws_order_number#78, 5), ENSURE_REQUIREMENTS, [plan_id=11] +(71) CometSort +Input [7]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_order_number#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Arguments: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_order_number#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72], [ws_item_sk#66 ASC NULLS FIRST, ws_order_number#69 ASC NULLS FIRST] -(75) Sort [codegen id : 22] -Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Arguments: [ws_item_sk#75 ASC NULLS FIRST, ws_order_number#78 ASC NULLS FIRST], false, 0 - -(76) Scan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +(72) Scan parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77, wr_returned_date_sk#78] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct -(77) CometFilter -Input [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] -Condition : (isnotnull(wr_item_sk#83) AND isnotnull(wr_order_number#84)) - -(78) CometProject -Input [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] -Arguments: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86], [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] +(73) CometFilter +Input [5]: [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77, wr_returned_date_sk#78] +Condition : (isnotnull(wr_item_sk#74) AND isnotnull(wr_order_number#75)) -(79) ColumnarToRow [codegen id : 23] -Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] +(74) CometProject +Input [5]: [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77, wr_returned_date_sk#78] +Arguments: [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77], [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77] -(80) Exchange -Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -Arguments: hashpartitioning(wr_item_sk#83, wr_order_number#84, 5), ENSURE_REQUIREMENTS, [plan_id=12] +(75) CometColumnarExchange +Input [4]: [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77] +Arguments: hashpartitioning(wr_item_sk#74, wr_order_number#75, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(81) Sort [codegen id : 24] -Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -Arguments: [wr_item_sk#83 ASC NULLS FIRST, wr_order_number#84 ASC NULLS FIRST], false, 0 +(76) CometSort +Input [4]: [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77] +Arguments: [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77], [wr_item_sk#74 ASC NULLS FIRST, wr_order_number#75 ASC NULLS FIRST] -(82) SortMergeJoin [codegen id : 29] -Left keys [2]: [ws_item_sk#75, ws_order_number#78] -Right keys [2]: [wr_item_sk#83, wr_order_number#84] -Join type: LeftOuter -Join condition: None +(77) CometSortMergeJoin +Left output [7]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_order_number#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Right output [4]: [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77] +Arguments: [ws_item_sk#66, ws_order_number#69], [wr_item_sk#74, wr_order_number#75], LeftOuter -(83) Project [codegen id : 29] -Output [8]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86] -Input [11]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] +(78) CometProject +Input [11]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_order_number#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77] +Arguments: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, wr_return_amt#76, wr_net_loss#77], [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, wr_return_amt#76, wr_net_loss#77] -(84) ReusedExchange [Reuses operator id: 112] -Output [1]: [d_date_sk#88] +(79) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#79] -(85) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_sold_date_sk#81] -Right keys [1]: [d_date_sk#88] -Join type: Inner -Join condition: None +(80) CometBroadcastHashJoin +Left output [8]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, wr_return_amt#76, wr_net_loss#77] +Right output [1]: [d_date_sk#79] +Arguments: [ws_sold_date_sk#72], [d_date_sk#79], Inner, BuildRight -(86) Project [codegen id : 29] -Output [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86] -Input [9]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86, d_date_sk#88] +(81) CometProject +Input [9]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, wr_return_amt#76, wr_net_loss#77, d_date_sk#79] +Arguments: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77], [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77] -(87) Scan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#89, web_site_id#90] +(82) Scan parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#80, web_site_id#81] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(88) CometFilter -Input [2]: [web_site_sk#89, web_site_id#90] -Condition : isnotnull(web_site_sk#89) - -(89) ColumnarToRow [codegen id : 26] -Input [2]: [web_site_sk#89, web_site_id#90] - -(90) BroadcastExchange -Input [2]: [web_site_sk#89, web_site_id#90] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] - -(91) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_web_site_sk#76] -Right keys [1]: [web_site_sk#89] -Join type: Inner -Join condition: None - -(92) Project [codegen id : 29] -Output [7]: [ws_item_sk#75, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] -Input [9]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_sk#89, web_site_id#90] - -(93) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#91] - -(94) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_item_sk#75] -Right keys [1]: [i_item_sk#91] -Join type: Inner -Join condition: None - -(95) Project [codegen id : 29] -Output [6]: [ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] -Input [8]: [ws_item_sk#75, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90, i_item_sk#91] - -(96) ReusedExchange [Reuses operator id: 34] -Output [1]: [p_promo_sk#92] - -(97) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_promo_sk#77] -Right keys [1]: [p_promo_sk#92] -Join type: Inner -Join condition: None - -(98) Project [codegen id : 29] -Output [5]: [ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] -Input [7]: [ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90, p_promo_sk#92] - -(99) HashAggregate [codegen id : 29] -Input [5]: [ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] -Keys [1]: [web_site_id#90] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#79)), partial_sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))] -Aggregate Attributes [5]: [sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] -Results [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] - -(100) Exchange -Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -Arguments: hashpartitioning(web_site_id#90, 5), ENSURE_REQUIREMENTS, [plan_id=14] - -(101) HashAggregate [codegen id : 30] -Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -Keys [1]: [web_site_id#90] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#79)), sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00)), sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#79))#103, sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00))#104, sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))#105] -Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#79))#103,17,2) AS sales#106, sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00))#104 AS returns#107, sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))#105 AS profit#108, web channel AS channel#109, concat(web_site, web_site_id#90) AS id#110] - -(102) Union - -(103) Expand [codegen id : 31] -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#111, id#112, spark_grouping_id#113] - -(104) HashAggregate [codegen id : 31] -Input [6]: [sales#34, returns#35, profit#36, channel#111, id#112, spark_grouping_id#113] -Keys [3]: [channel#111, id#112, spark_grouping_id#113] -Functions [3]: [partial_sum(sales#34), partial_sum(returns#35), partial_sum(profit#36)] -Aggregate Attributes [6]: [sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] -Results [9]: [channel#111, id#112, spark_grouping_id#113, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] - -(105) Exchange -Input [9]: [channel#111, id#112, spark_grouping_id#113, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] -Arguments: hashpartitioning(channel#111, id#112, spark_grouping_id#113, 5), ENSURE_REQUIREMENTS, [plan_id=15] - -(106) HashAggregate [codegen id : 32] -Input [9]: [channel#111, id#112, spark_grouping_id#113, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] -Keys [3]: [channel#111, id#112, spark_grouping_id#113] -Functions [3]: [sum(sales#34), sum(returns#35), sum(profit#36)] -Aggregate Attributes [3]: [sum(sales#34)#126, sum(returns#35)#127, sum(profit#36)#128] -Results [5]: [channel#111, id#112, sum(sales#34)#126 AS sales#129, sum(returns#35)#127 AS returns#130, sum(profit#36)#128 AS profit#131] - -(107) TakeOrderedAndProject -Input [5]: [channel#111, id#112, sales#129, returns#130, profit#131] -Arguments: 100, [channel#111 ASC NULLS FIRST, id#112 ASC NULLS FIRST], [channel#111, id#112, sales#129, returns#130, profit#131] +(83) CometFilter +Input [2]: [web_site_sk#80, web_site_id#81] +Condition : isnotnull(web_site_sk#80) + +(84) CometBroadcastExchange +Input [2]: [web_site_sk#80, web_site_id#81] +Arguments: [web_site_sk#80, web_site_id#81] + +(85) CometBroadcastHashJoin +Left output [7]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77] +Right output [2]: [web_site_sk#80, web_site_id#81] +Arguments: [ws_web_site_sk#67], [web_site_sk#80], Inner, BuildRight + +(86) CometProject +Input [9]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_sk#80, web_site_id#81] +Arguments: [ws_item_sk#66, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81], [ws_item_sk#66, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81] + +(87) ReusedExchange [Reuses operator id: 26] +Output [1]: [i_item_sk#82] + +(88) CometBroadcastHashJoin +Left output [7]: [ws_item_sk#66, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81] +Right output [1]: [i_item_sk#82] +Arguments: [ws_item_sk#66], [i_item_sk#82], Inner, BuildRight + +(89) CometProject +Input [8]: [ws_item_sk#66, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81, i_item_sk#82] +Arguments: [ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81], [ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81] + +(90) ReusedExchange [Reuses operator id: 32] +Output [1]: [p_promo_sk#83] + +(91) CometBroadcastHashJoin +Left output [6]: [ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81] +Right output [1]: [p_promo_sk#83] +Arguments: [ws_promo_sk#68], [p_promo_sk#83], Inner, BuildRight + +(92) CometProject +Input [7]: [ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81, p_promo_sk#83] +Arguments: [ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81], [ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81] + +(93) CometHashAggregate +Input [5]: [ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81] +Keys [1]: [web_site_id#81] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(coalesce(cast(wr_return_amt#76 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#71 - coalesce(cast(wr_net_loss#77 as decimal(12,2)), 0.00)))] + +(94) CometColumnarExchange +Input [6]: [web_site_id#81, sum#84, sum#85, isEmpty#86, sum#87, isEmpty#88] +Arguments: hashpartitioning(web_site_id#81, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(95) ColumnarToRow [codegen id : 3] +Input [6]: [web_site_id#81, sum#84, sum#85, isEmpty#86, sum#87, isEmpty#88] + +(96) HashAggregate [codegen id : 3] +Input [6]: [web_site_id#81, sum#84, sum#85, isEmpty#86, sum#87, isEmpty#88] +Keys [1]: [web_site_id#81] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#70)), sum(coalesce(cast(wr_return_amt#76 as decimal(12,2)), 0.00)), sum((ws_net_profit#71 - coalesce(cast(wr_net_loss#77 as decimal(12,2)), 0.00)))] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#70))#89, sum(coalesce(cast(wr_return_amt#76 as decimal(12,2)), 0.00))#90, sum((ws_net_profit#71 - coalesce(cast(wr_net_loss#77 as decimal(12,2)), 0.00)))#91] +Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#70))#89,17,2) AS sales#92, sum(coalesce(cast(wr_return_amt#76 as decimal(12,2)), 0.00))#90 AS returns#93, sum((ws_net_profit#71 - coalesce(cast(wr_net_loss#77 as decimal(12,2)), 0.00)))#91 AS profit#94, web channel AS channel#95, concat(web_site, web_site_id#81) AS id#96] + +(97) Union + +(98) Expand [codegen id : 4] +Input [5]: [sales#30, returns#31, profit#32, channel#33, id#34] +Arguments: [[sales#30, returns#31, profit#32, channel#33, id#34, 0], [sales#30, returns#31, profit#32, channel#33, null, 1], [sales#30, returns#31, profit#32, null, null, 3]], [sales#30, returns#31, profit#32, channel#97, id#98, spark_grouping_id#99] + +(99) HashAggregate [codegen id : 4] +Input [6]: [sales#30, returns#31, profit#32, channel#97, id#98, spark_grouping_id#99] +Keys [3]: [channel#97, id#98, spark_grouping_id#99] +Functions [3]: [partial_sum(sales#30), partial_sum(returns#31), partial_sum(profit#32)] +Aggregate Attributes [6]: [sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] +Results [9]: [channel#97, id#98, spark_grouping_id#99, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] + +(100) RowToColumnar +Input [9]: [channel#97, id#98, spark_grouping_id#99, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] + +(101) CometColumnarExchange +Input [9]: [channel#97, id#98, spark_grouping_id#99, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] +Arguments: hashpartitioning(channel#97, id#98, spark_grouping_id#99, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(102) ColumnarToRow [codegen id : 5] +Input [9]: [channel#97, id#98, spark_grouping_id#99, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] + +(103) HashAggregate [codegen id : 5] +Input [9]: [channel#97, id#98, spark_grouping_id#99, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] +Keys [3]: [channel#97, id#98, spark_grouping_id#99] +Functions [3]: [sum(sales#30), sum(returns#31), sum(profit#32)] +Aggregate Attributes [3]: [sum(sales#30)#112, sum(returns#31)#113, sum(profit#32)#114] +Results [5]: [channel#97, id#98, sum(sales#30)#112 AS sales#115, sum(returns#31)#113 AS returns#116, sum(profit#32)#114 AS profit#117] + +(104) TakeOrderedAndProject +Input [5]: [channel#97, id#98, sales#115, returns#116, profit#117] +Arguments: 100, [channel#97 ASC NULLS FIRST, id#98 ASC NULLS FIRST], [channel#97, id#98, sales#115, returns#116, profit#117] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (112) -+- * ColumnarToRow (111) - +- CometProject (110) - +- CometFilter (109) - +- CometScan parquet spark_catalog.default.date_dim (108) +BroadcastExchange (109) ++- * ColumnarToRow (108) + +- CometProject (107) + +- CometFilter (106) + +- CometScan parquet spark_catalog.default.date_dim (105) -(108) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#132] +(105) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#15] 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-22), IsNotNull(d_date_sk)] ReadSchema: struct -(109) CometFilter -Input [2]: [d_date_sk#14, d_date#132] -Condition : (((isnotnull(d_date#132) AND (d_date#132 >= 2000-08-23)) AND (d_date#132 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) +(106) CometFilter +Input [2]: [d_date_sk#14, d_date#15] +Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#15 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) -(110) CometProject -Input [2]: [d_date_sk#14, d_date#132] +(107) CometProject +Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(111) ColumnarToRow [codegen id : 1] +(108) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(112) BroadcastExchange +(109) BroadcastExchange Input [1]: [d_date_sk#14] -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=11] -Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 71 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#72 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt index cdc8dc486..dfcc35f6e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt @@ -1,182 +1,124 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (32) + WholeStageCodegen (5) 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 (31) - 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 (10) - HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [s_store_id] #2 - WholeStageCodegen (9) - HashAggregate [s_store_id,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - SortMergeJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - WholeStageCodegen (2) - Sort [ss_item_sk,ss_ticket_number] - InputAdapter - Exchange [ss_item_sk,ss_ticket_number] #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - WholeStageCodegen (4) - Sort [sr_item_sk,sr_ticket_number] - InputAdapter - Exchange [sr_item_sk,sr_ticket_number] #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - 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 (7) - ColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - WholeStageCodegen (20) - HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [cp_catalog_page_id] #9 - WholeStageCodegen (19) - HashAggregate [cp_catalog_page_id,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_catalog_page_sk,cp_catalog_page_sk] - Project [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] - InputAdapter - WholeStageCodegen (12) - Sort [cs_item_sk,cs_order_number] - InputAdapter - Exchange [cs_item_sk,cs_order_number] #10 - WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - WholeStageCodegen (14) - Sort [cr_item_sk,cr_order_number] - InputAdapter - Exchange [cr_item_sk,cr_order_number] #11 - WholeStageCodegen (13) - ColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - InputAdapter - ReusedExchange [i_item_sk] #7 - InputAdapter - ReusedExchange [p_promo_sk] #8 - WholeStageCodegen (30) - HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [web_site_id] #13 - WholeStageCodegen (29) - HashAggregate [web_site_id,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_promo_sk,p_promo_sk] - Project [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - SortMergeJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] - InputAdapter - WholeStageCodegen (22) - Sort [ws_item_sk,ws_order_number] - InputAdapter - Exchange [ws_item_sk,ws_order_number] #14 - WholeStageCodegen (21) - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - WholeStageCodegen (24) - Sort [wr_item_sk,wr_order_number] - InputAdapter - Exchange [wr_item_sk,wr_order_number] #15 - WholeStageCodegen (23) - ColumnarToRow - InputAdapter - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #16 - WholeStageCodegen (26) - ColumnarToRow - InputAdapter - CometFilter [web_site_sk,web_site_id] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - InputAdapter - ReusedExchange [i_item_sk] #7 - InputAdapter - ReusedExchange [p_promo_sk] #8 + ColumnarToRow + InputAdapter + CometColumnarExchange [channel,id,spark_grouping_id] #1 + RowToColumnar + WholeStageCodegen (4) + 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 (1) + HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [s_store_id] #2 + CometHashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] + CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] + CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] + CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] + CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk,ss_ticket_number] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometColumnarExchange [sr_item_sk,sr_ticket_number] #5 + CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [s_store_sk,s_store_id] #7 + CometFilter [s_store_sk,s_store_id] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange [i_item_sk] #8 + CometProject [i_item_sk] + CometFilter [i_item_sk,i_current_price] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] + CometBroadcastExchange [p_promo_sk] #9 + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] + CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] + WholeStageCodegen (2) + HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [cp_catalog_page_id] #10 + CometHashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] + CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] + CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] + CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] + CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometColumnarExchange [cs_item_sk,cs_order_number] #11 + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometColumnarExchange [cr_item_sk,cr_order_number] #12 + CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #13 + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + ReusedExchange [i_item_sk] #8 + ReusedExchange [p_promo_sk] #9 + WholeStageCodegen (3) + HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [web_site_id] #14 + CometHashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] + CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] + CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] + CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] + CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometColumnarExchange [ws_item_sk,ws_order_number] #15 + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometColumnarExchange [wr_item_sk,wr_order_number] #16 + CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + CometBroadcastExchange [web_site_sk,web_site_id] #17 + CometFilter [web_site_sk,web_site_id] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + ReusedExchange [i_item_sk] #8 + ReusedExchange [p_promo_sk] #9 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 67c8af083..97abada79 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,54 +1,51 @@ == Physical Plan == -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) +* ColumnarToRow (47) ++- CometTakeOrderedAndProject (46) + +- CometProject (45) + +- CometBroadcastHashJoin (44) + :- CometProject (40) + : +- CometBroadcastHashJoin (39) + : :- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometFilter (17) + : : : +- CometHashAggregate (16) + : : : +- CometColumnarExchange (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) + : : +- CometBroadcastExchange (33) + : : +- CometFilter (32) + : : +- CometHashAggregate (31) + : : +- CometColumnarExchange (30) + : : +- CometHashAggregate (29) + : : +- CometHashAggregate (28) + : : +- CometColumnarExchange (27) + : : +- CometHashAggregate (26) + : : +- CometProject (25) + : : +- CometBroadcastHashJoin (24) + : : :- CometProject (22) + : : : +- CometBroadcastHashJoin (21) + : : : :- CometFilter (19) + : : : : +- CometScan parquet spark_catalog.default.catalog_returns (18) + : : : +- ReusedExchange (20) + : : +- ReusedExchange (23) + : +- CometBroadcastExchange (38) + : +- CometFilter (37) + : +- CometScan parquet spark_catalog.default.customer (36) + +- CometBroadcastExchange (43) + +- CometFilter (42) + +- CometScan parquet spark_catalog.default.customer_address (41) (1) Scan parquet spark_catalog.default.catalog_returns @@ -120,207 +117,187 @@ 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))] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(17) HashAggregate [codegen id : 7] +(16) CometHashAggregate 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#9 AS ctr_state#13, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#11,17,2) AS ctr_total_return#14] -(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) CometFilter +Input [3]: [ctr_customer_sk#11, ctr_state#12, ctr_total_return#13] +Condition : isnotnull(ctr_total_return#13) -(19) Scan parquet spark_catalog.default.catalog_returns +(18) 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 [] -PartitionFilters: [isnotnull(cr_returned_date_sk#4), dynamicpruningexpression(cr_returned_date_sk#4 IN dynamicpruning#15)] +PartitionFilters: [isnotnull(cr_returned_date_sk#4), dynamicpruningexpression(cr_returned_date_sk#4 IN dynamicpruning#14)] PushedFilters: [IsNotNull(cr_returning_addr_sk)] ReadSchema: struct -(20) CometFilter +(19) 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) -(21) ReusedExchange [Reuses operator id: 6] +(20) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#6] -(22) CometBroadcastHashJoin +(21) 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 -(23) CometProject +(22) 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] -(24) ReusedExchange [Reuses operator id: 11] +(23) ReusedExchange [Reuses operator id: 11] Output [2]: [ca_address_sk#8, ca_state#9] -(25) CometBroadcastHashJoin +(24) 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 -(26) CometProject +(25) 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] -(27) CometHashAggregate +(26) 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))] -(28) ColumnarToRow [codegen id : 2] -Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#16] - -(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] +(27) CometColumnarExchange +Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#15] +Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(30) HashAggregate [codegen id : 3] -Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#16] +(28) CometHashAggregate +Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#15] 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#9 AS ctr_state#13, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#11,17,2) AS ctr_total_return#14] - -(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#17, count#18] -Results [3]: [ctr_state#13, sum#19, count#20] - -(32) Exchange -Input [3]: [ctr_state#13, sum#19, count#20] -Arguments: hashpartitioning(ctr_state#13, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(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)#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] - -(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) - -(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] - -(36) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ctr_state#13] -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)#22) - -(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)#22, ctr_state#13#23] - -(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] + +(29) CometHashAggregate +Input [2]: [ctr_state#12, ctr_total_return#13] +Keys [1]: [ctr_state#12] +Functions [1]: [partial_avg(ctr_total_return#13)] + +(30) CometColumnarExchange +Input [3]: [ctr_state#12, sum#16, count#17] +Arguments: hashpartitioning(ctr_state#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(31) CometHashAggregate +Input [3]: [ctr_state#12, sum#16, count#17] +Keys [1]: [ctr_state#12] +Functions [1]: [avg(ctr_total_return#13)] + +(32) CometFilter +Input [2]: [(avg(ctr_total_return) * 1.2)#18, ctr_state#12#19] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#18) + +(33) CometBroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#18, ctr_state#12#19] +Arguments: [(avg(ctr_total_return) * 1.2)#18, ctr_state#12#19] + +(34) CometBroadcastHashJoin +Left output [3]: [ctr_customer_sk#11, ctr_state#12, ctr_total_return#13] +Right output [2]: [(avg(ctr_total_return) * 1.2)#18, ctr_state#12#19] +Arguments: [ctr_state#12], [ctr_state#12#19], Inner, (cast(ctr_total_return#13 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#18), BuildRight + +(35) CometProject +Input [5]: [ctr_customer_sk#11, ctr_state#12, ctr_total_return#13, (avg(ctr_total_return) * 1.2)#18, ctr_state#12#19] +Arguments: [ctr_customer_sk#11, ctr_total_return#13], [ctr_customer_sk#11, ctr_total_return#13] + +(36) Scan parquet spark_catalog.default.customer +Output [6]: [c_customer_sk#20, c_customer_id#21, c_current_addr_sk#22, c_salutation#23, c_first_name#24, c_last_name#25] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(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)) - -(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] +(37) CometFilter +Input [6]: [c_customer_sk#20, c_customer_id#21, c_current_addr_sk#22, c_salutation#23, c_first_name#24, c_last_name#25] +Condition : (isnotnull(c_customer_sk#20) AND isnotnull(c_current_addr_sk#22)) -(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] +(38) CometBroadcastExchange +Input [6]: [c_customer_sk#20, c_customer_id#21, c_current_addr_sk#22, c_salutation#23, c_first_name#24, c_last_name#25] +Arguments: [c_customer_sk#20, c_customer_id#21, c_current_addr_sk#22, c_salutation#23, c_first_name#24, c_last_name#25] -(42) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ctr_customer_sk#12] -Right keys [1]: [c_customer_sk#24] -Join type: Inner -Join condition: None +(39) CometBroadcastHashJoin +Left output [2]: [ctr_customer_sk#11, ctr_total_return#13] +Right output [6]: [c_customer_sk#20, c_customer_id#21, c_current_addr_sk#22, c_salutation#23, c_first_name#24, c_last_name#25] +Arguments: [ctr_customer_sk#11], [c_customer_sk#20], Inner, BuildRight -(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] +(40) CometProject +Input [8]: [ctr_customer_sk#11, ctr_total_return#13, c_customer_sk#20, c_customer_id#21, c_current_addr_sk#22, c_salutation#23, c_first_name#24, c_last_name#25] +Arguments: [ctr_total_return#13, c_customer_id#21, c_current_addr_sk#22, c_salutation#23, c_first_name#24, c_last_name#25], [ctr_total_return#13, c_customer_id#21, c_current_addr_sk#22, c_salutation#23, c_first_name#24, c_last_name#25] -(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] +(41) Scan parquet spark_catalog.default.customer_address +Output [12]: [ca_address_sk#26, ca_street_number#27, ca_street_name#28, ca_street_type#29, ca_suite_number#30, ca_city#31, ca_county#32, ca_state#33, ca_zip#34, ca_country#35, ca_gmt_offset#36, ca_location_type#37] 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 -(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)) +(42) CometFilter +Input [12]: [ca_address_sk#26, ca_street_number#27, ca_street_name#28, ca_street_type#29, ca_suite_number#30, ca_city#31, ca_county#32, ca_state#33, ca_zip#34, ca_country#35, ca_gmt_offset#36, ca_location_type#37] +Condition : ((isnotnull(ca_state#33) AND (ca_state#33 = GA)) AND isnotnull(ca_address_sk#26)) -(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] +(43) CometBroadcastExchange +Input [12]: [ca_address_sk#26, ca_street_number#27, ca_street_name#28, ca_street_type#29, ca_suite_number#30, ca_city#31, ca_county#32, ca_state#33, ca_zip#34, ca_country#35, ca_gmt_offset#36, ca_location_type#37] +Arguments: [ca_address_sk#26, ca_street_number#27, ca_street_name#28, ca_street_type#29, ca_suite_number#30, ca_city#31, ca_county#32, ca_state#33, ca_zip#34, ca_country#35, ca_gmt_offset#36, ca_location_type#37] -(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] +(44) CometBroadcastHashJoin +Left output [6]: [ctr_total_return#13, c_customer_id#21, c_current_addr_sk#22, c_salutation#23, c_first_name#24, c_last_name#25] +Right output [12]: [ca_address_sk#26, ca_street_number#27, ca_street_name#28, ca_street_type#29, ca_suite_number#30, ca_city#31, ca_county#32, ca_state#33, ca_zip#34, ca_country#35, ca_gmt_offset#36, ca_location_type#37] +Arguments: [c_current_addr_sk#22], [ca_address_sk#26], Inner, BuildRight -(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 +(45) CometProject +Input [18]: [ctr_total_return#13, c_customer_id#21, c_current_addr_sk#22, c_salutation#23, c_first_name#24, c_last_name#25, ca_address_sk#26, ca_street_number#27, ca_street_name#28, ca_street_type#29, ca_suite_number#30, ca_city#31, ca_county#32, ca_state#33, ca_zip#34, ca_country#35, ca_gmt_offset#36, ca_location_type#37] +Arguments: [c_customer_id#21, c_salutation#23, c_first_name#24, c_last_name#25, ca_street_number#27, ca_street_name#28, ca_street_type#29, ca_suite_number#30, ca_city#31, ca_county#32, ca_state#33, ca_zip#34, ca_country#35, ca_gmt_offset#36, ca_location_type#37, ctr_total_return#13], [c_customer_id#21, c_salutation#23, c_first_name#24, c_last_name#25, ca_street_number#27, ca_street_name#28, ca_street_type#29, ca_suite_number#30, ca_city#31, ca_county#32, ca_state#33, ca_zip#34, ca_country#35, ca_gmt_offset#36, ca_location_type#37, ctr_total_return#13] -(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] +(46) CometTakeOrderedAndProject +Input [16]: [c_customer_id#21, c_salutation#23, c_first_name#24, c_last_name#25, ca_street_number#27, ca_street_name#28, ca_street_type#29, ca_suite_number#30, ca_city#31, ca_county#32, ca_state#33, ca_zip#34, ca_country#35, ca_gmt_offset#36, ca_location_type#37, ctr_total_return#13] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#21 ASC NULLS FIRST,c_salutation#23 ASC NULLS FIRST,c_first_name#24 ASC NULLS FIRST,c_last_name#25 ASC NULLS FIRST,ca_street_number#27 ASC NULLS FIRST,ca_street_name#28 ASC NULLS FIRST,ca_street_type#29 ASC NULLS FIRST,ca_suite_number#30 ASC NULLS FIRST,ca_city#31 ASC NULLS FIRST,ca_county#32 ASC NULLS FIRST,ca_state#33 ASC NULLS FIRST,ca_zip#34 ASC NULLS FIRST,ca_country#35 ASC NULLS FIRST,ca_gmt_offset#36 ASC NULLS FIRST,ca_location_type#37 ASC NULLS FIRST,ctr_total_return#13 ASC NULLS FIRST], output=[c_customer_id#21,c_salutation#23,c_first_name#24,c_last_name#25,ca_street_number#27,ca_street_name#28,ca_street_type#29,ca_suite_number#30,ca_city#31,ca_county#32,ca_state#33,ca_zip#34,ca_country#35,ca_gmt_offset#36,ca_location_type#37,ctr_total_return#13]), [c_customer_id#21, c_salutation#23, c_first_name#24, c_last_name#25, ca_street_number#27, ca_street_name#28, ca_street_type#29, ca_suite_number#30, ca_city#31, ca_county#32, ca_state#33, ca_zip#34, ca_country#35, ca_gmt_offset#36, ca_location_type#37, ctr_total_return#13], 100, [c_customer_id#21 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_last_name#25 ASC NULLS FIRST, ca_street_number#27 ASC NULLS FIRST, ca_street_name#28 ASC NULLS FIRST, ca_street_type#29 ASC NULLS FIRST, ca_suite_number#30 ASC NULLS FIRST, ca_city#31 ASC NULLS FIRST, ca_county#32 ASC NULLS FIRST, ca_state#33 ASC NULLS FIRST, ca_zip#34 ASC NULLS FIRST, ca_country#35 ASC NULLS FIRST, ca_gmt_offset#36 ASC NULLS FIRST, ca_location_type#37 ASC NULLS FIRST, ctr_total_return#13 ASC NULLS FIRST], [c_customer_id#21, c_salutation#23, c_first_name#24, c_last_name#25, ca_street_number#27, ca_street_name#28, ca_street_type#29, ca_suite_number#30, ca_city#31, ca_county#32, ca_state#33, ca_zip#34, ca_country#35, ca_gmt_offset#36, ca_location_type#37, ctr_total_return#13] -(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] +(47) ColumnarToRow [codegen id : 1] +Input [16]: [c_customer_id#21, c_salutation#23, c_first_name#24, c_last_name#25, ca_street_number#27, ca_street_name#28, ca_street_type#29, ca_suite_number#30, ca_city#31, ca_county#32, ca_state#33, ca_zip#34, ca_country#35, ca_gmt_offset#36, ca_location_type#37, ctr_total_return#13] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (55) -+- * ColumnarToRow (54) - +- CometProject (53) - +- CometFilter (52) - +- CometScan parquet spark_catalog.default.date_dim (51) +BroadcastExchange (52) ++- * ColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) -(51) Scan parquet spark_catalog.default.date_dim +(48) 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 -(52) CometFilter +(49) 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)) -(53) CometProject +(50) CometProject Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(54) ColumnarToRow [codegen id : 1] +(51) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(55) BroadcastExchange +(52) BroadcastExchange Input [1]: [d_date_sk#6] -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=4] -Subquery:2 Hosting operator id = 19 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 18 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 e576a079b..62f7d5462 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,76 +1,58 @@ -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 (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] - BroadcastHashJoin [ctr_customer_sk,c_customer_sk] - Project [ctr_customer_sk,ctr_total_return] - BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] - Filter [ctr_total_return] - HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_customer_sk,ctr_state,ctr_total_return,sum] - InputAdapter - Exchange [cr_returning_customer_sk,ca_state] #1 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [cr_returning_customer_sk,ca_state,sum,cr_return_amt_inc_tax] - CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] - CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] - CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [ca_address_sk,ca_state] #4 - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - 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] #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] #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometHashAggregate [cr_returning_customer_sk,ca_state,sum,cr_return_amt_inc_tax] - CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] - CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] - CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_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 - ReusedExchange [d_date_sk] #3 - ReusedExchange [ca_address_sk,ca_state] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - 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 #9 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [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] + CometProject [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] + CometBroadcastHashJoin [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] + CometProject [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + CometBroadcastHashJoin [ctr_customer_sk,ctr_total_return,c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + CometProject [ctr_customer_sk,ctr_total_return] + CometBroadcastHashJoin [ctr_customer_sk,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_state] + CometFilter [ctr_customer_sk,ctr_state,ctr_total_return] + CometHashAggregate [ctr_customer_sk,ctr_state,ctr_total_return,cr_returning_customer_sk,ca_state,sum,sum(UnscaledValue(cr_return_amt_inc_tax))] + CometColumnarExchange [cr_returning_customer_sk,ca_state] #1 + CometHashAggregate [cr_returning_customer_sk,ca_state,sum,cr_return_amt_inc_tax] + CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] + CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] + CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [ca_address_sk,ca_state] #4 + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_state] #5 + CometFilter [(avg(ctr_total_return) * 1.2),ctr_state] + CometHashAggregate [(avg(ctr_total_return) * 1.2),ctr_state,ctr_state,sum,count,avg(ctr_total_return)] + CometColumnarExchange [ctr_state] #6 + CometHashAggregate [ctr_state,sum,count,ctr_total_return] + CometHashAggregate [ctr_state,ctr_total_return,cr_returning_customer_sk,ca_state,sum,sum(UnscaledValue(cr_return_amt_inc_tax))] + CometColumnarExchange [cr_returning_customer_sk,ca_state] #7 + CometHashAggregate [cr_returning_customer_sk,ca_state,sum,cr_return_amt_inc_tax] + CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] + CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] + CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_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 + ReusedExchange [d_date_sk] #3 + ReusedExchange [ca_address_sk,ca_state] #4 + CometBroadcastExchange [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] #8 + CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] #9 + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] diff --git a/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 00e38f694..2519fbc97 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,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (26) -+- * HashAggregate (25) - +- Exchange (24) - +- * ColumnarToRow (23) +* ColumnarToRow (26) ++- CometTakeOrderedAndProject (25) + +- CometHashAggregate (24) + +- CometColumnarExchange (23) +- CometHashAggregate (22) +- CometProject (21) +- CometBroadcastHashJoin (20) @@ -132,23 +132,21 @@ 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: [] -(23) ColumnarToRow [codegen id : 1] +(23) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(24) 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] -(26) TakeOrderedAndProject +(25) CometTakeOrderedAndProject +Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#2 ASC NULLS FIRST], output=[i_item_id#2,i_item_desc#3,i_current_price#4]), [i_item_id#2, i_item_desc#3, i_current_price#4], 100, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4] + +(26) ColumnarToRow [codegen id : 1] 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 ===== 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 71a1c8e17..869073de1 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,38 +1,36 @@ -TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - 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 (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,i_item_id,i_item_desc,i_current_price,ss_item_sk] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2 - CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,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,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] - CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 - CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_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_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometProject [ss_item_sk] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] + CometHashAggregate [i_item_id,i_item_desc,i_current_price] + CometColumnarExchange [i_item_id,i_item_desc,i_current_price] #1 + CometHashAggregate [i_item_id,i_item_desc,i_current_price] + CometProject [i_item_id,i_item_desc,i_current_price] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,ss_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2 + CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,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,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] + CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] + CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 + CometProject [inv_item_sk,inv_date_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_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_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [ss_item_sk] + CometFilter [ss_item_sk,ss_sold_date_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 1838b9a77..972a58976 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,54 +1,52 @@ == Physical Plan == -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) +* ColumnarToRow (48) ++- CometTakeOrderedAndProject (47) + +- CometProject (46) + +- CometBroadcastHashJoin (45) + :- CometProject (32) + : +- CometBroadcastHashJoin (31) + : :- CometHashAggregate (18) + : : +- CometColumnarExchange (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) + : +- CometBroadcastExchange (30) + : +- CometHashAggregate (29) + : +- CometColumnarExchange (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) + +- CometBroadcastExchange (44) + +- CometHashAggregate (43) + +- CometColumnarExchange (42) + +- CometHashAggregate (41) + +- CometProject (40) + +- CometBroadcastHashJoin (39) + :- CometProject (37) + : +- CometBroadcastHashJoin (36) + : :- CometFilter (34) + : : +- CometScan parquet spark_catalog.default.web_returns (33) + : +- ReusedExchange (35) + +- ReusedExchange (38) (1) Scan parquet spark_catalog.default.store_returns @@ -98,7 +96,7 @@ ReadSchema: struct Input [2]: [d_date_sk#7, d_date#8] Condition : isnotnull(d_date_sk#7) -(10) ReusedExchange [Reuses operator id: 60] +(10) ReusedExchange [Reuses operator id: 58] Output [1]: [d_date#9] (11) CometBroadcastHashJoin @@ -128,247 +126,233 @@ Input [2]: [sr_return_quantity#2, i_item_id#6] Keys [1]: [i_item_id#6] Functions [1]: [partial_sum(sr_return_quantity#2)] -(17) ColumnarToRow [codegen id : 1] +(17) CometColumnarExchange Input [2]: [i_item_id#6, sum#10] +Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(18) Exchange -Input [2]: [i_item_id#6, sum#10] -Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [plan_id=1] - -(19) HashAggregate [codegen id : 6] +(18) CometHashAggregate 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)#11] -Results [2]: [i_item_id#6 AS item_id#12, sum(sr_return_quantity#2)#11 AS sr_item_qty#13] -(20) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#14, cr_return_quantity#15, cr_returned_date_sk#16] +(19) Scan parquet spark_catalog.default.catalog_returns +Output [3]: [cr_item_sk#11, cr_return_quantity#12, cr_returned_date_sk#13] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#16), dynamicpruningexpression(cr_returned_date_sk#16 IN dynamicpruning#17)] +PartitionFilters: [isnotnull(cr_returned_date_sk#13), dynamicpruningexpression(cr_returned_date_sk#13 IN dynamicpruning#14)] PushedFilters: [IsNotNull(cr_item_sk)] ReadSchema: struct -(21) CometFilter -Input [3]: [cr_item_sk#14, cr_return_quantity#15, cr_returned_date_sk#16] -Condition : isnotnull(cr_item_sk#14) - -(22) ReusedExchange [Reuses operator id: 5] -Output [2]: [i_item_sk#18, i_item_id#19] - -(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 - -(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] - -(25) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#20] - -(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 - -(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)] - -(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=3] - -(33) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [item_id#12] -Right keys [1]: [item_id#23] -Join type: Inner -Join condition: None - -(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] - -(35) Scan parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] +(20) CometFilter +Input [3]: [cr_item_sk#11, cr_return_quantity#12, cr_returned_date_sk#13] +Condition : isnotnull(cr_item_sk#11) + +(21) ReusedExchange [Reuses operator id: 5] +Output [2]: [i_item_sk#15, i_item_id#16] + +(22) CometBroadcastHashJoin +Left output [3]: [cr_item_sk#11, cr_return_quantity#12, cr_returned_date_sk#13] +Right output [2]: [i_item_sk#15, i_item_id#16] +Arguments: [cr_item_sk#11], [i_item_sk#15], Inner, BuildRight + +(23) CometProject +Input [5]: [cr_item_sk#11, cr_return_quantity#12, cr_returned_date_sk#13, i_item_sk#15, i_item_id#16] +Arguments: [cr_return_quantity#12, cr_returned_date_sk#13, i_item_id#16], [cr_return_quantity#12, cr_returned_date_sk#13, i_item_id#16] + +(24) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_date_sk#17] + +(25) CometBroadcastHashJoin +Left output [3]: [cr_return_quantity#12, cr_returned_date_sk#13, i_item_id#16] +Right output [1]: [d_date_sk#17] +Arguments: [cr_returned_date_sk#13], [d_date_sk#17], Inner, BuildRight + +(26) CometProject +Input [4]: [cr_return_quantity#12, cr_returned_date_sk#13, i_item_id#16, d_date_sk#17] +Arguments: [cr_return_quantity#12, i_item_id#16], [cr_return_quantity#12, i_item_id#16] + +(27) CometHashAggregate +Input [2]: [cr_return_quantity#12, i_item_id#16] +Keys [1]: [i_item_id#16] +Functions [1]: [partial_sum(cr_return_quantity#12)] + +(28) CometColumnarExchange +Input [2]: [i_item_id#16, sum#18] +Arguments: hashpartitioning(i_item_id#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(29) CometHashAggregate +Input [2]: [i_item_id#16, sum#18] +Keys [1]: [i_item_id#16] +Functions [1]: [sum(cr_return_quantity#12)] + +(30) CometBroadcastExchange +Input [2]: [item_id#19, cr_item_qty#20] +Arguments: [item_id#19, cr_item_qty#20] + +(31) CometBroadcastHashJoin +Left output [2]: [item_id#21, sr_item_qty#22] +Right output [2]: [item_id#19, cr_item_qty#20] +Arguments: [item_id#21], [item_id#19], Inner, BuildRight + +(32) CometProject +Input [4]: [item_id#21, sr_item_qty#22, item_id#19, cr_item_qty#20] +Arguments: [item_id#21, sr_item_qty#22, cr_item_qty#20], [item_id#21, sr_item_qty#22, cr_item_qty#20] + +(33) Scan parquet spark_catalog.default.web_returns +Output [3]: [wr_item_sk#23, wr_return_quantity#24, wr_returned_date_sk#25] 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#25), dynamicpruningexpression(wr_returned_date_sk#25 IN dynamicpruning#26)] PushedFilters: [IsNotNull(wr_item_sk)] ReadSchema: struct -(36) CometFilter -Input [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] -Condition : isnotnull(wr_item_sk#25) +(34) CometFilter +Input [3]: [wr_item_sk#23, wr_return_quantity#24, wr_returned_date_sk#25] +Condition : isnotnull(wr_item_sk#23) + +(35) ReusedExchange [Reuses operator id: 5] +Output [2]: [i_item_sk#27, i_item_id#28] + +(36) CometBroadcastHashJoin +Left output [3]: [wr_item_sk#23, wr_return_quantity#24, wr_returned_date_sk#25] +Right output [2]: [i_item_sk#27, i_item_id#28] +Arguments: [wr_item_sk#23], [i_item_sk#27], Inner, BuildRight -(37) ReusedExchange [Reuses operator id: 5] -Output [2]: [i_item_sk#29, i_item_id#30] +(37) CometProject +Input [5]: [wr_item_sk#23, wr_return_quantity#24, wr_returned_date_sk#25, i_item_sk#27, i_item_id#28] +Arguments: [wr_return_quantity#24, wr_returned_date_sk#25, i_item_id#28], [wr_return_quantity#24, wr_returned_date_sk#25, i_item_id#28] -(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 +(38) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_date_sk#29] -(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] +(39) CometBroadcastHashJoin +Left output [3]: [wr_return_quantity#24, wr_returned_date_sk#25, i_item_id#28] +Right output [1]: [d_date_sk#29] +Arguments: [wr_returned_date_sk#25], [d_date_sk#29], Inner, BuildRight -(40) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#31] +(40) CometProject +Input [4]: [wr_return_quantity#24, wr_returned_date_sk#25, i_item_id#28, d_date_sk#29] +Arguments: [wr_return_quantity#24, i_item_id#28], [wr_return_quantity#24, i_item_id#28] -(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 +(41) CometHashAggregate +Input [2]: [wr_return_quantity#24, i_item_id#28] +Keys [1]: [i_item_id#28] +Functions [1]: [partial_sum(wr_return_quantity#24)] -(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] +(42) CometColumnarExchange +Input [2]: [i_item_id#28, sum#30] +Arguments: hashpartitioning(i_item_id#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (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)] - -(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] - -(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)#33] -Results [2]: [i_item_id#30 AS item_id#34, sum(wr_return_quantity#26)#33 AS wr_item_qty#35] - -(47) BroadcastExchange -Input [2]: [item_id#34, wr_item_qty#35] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] - -(48) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [item_id#12] -Right keys [1]: [item_id#34] -Join type: Inner -Join condition: None - -(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] - -(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] +Input [2]: [i_item_id#28, sum#30] +Keys [1]: [i_item_id#28] +Functions [1]: [sum(wr_return_quantity#24)] -===== Subqueries ===== +(44) CometBroadcastExchange +Input [2]: [item_id#31, wr_item_qty#32] +Arguments: [item_id#31, wr_item_qty#32] -Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 -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) +(45) CometBroadcastHashJoin +Left output [3]: [item_id#21, sr_item_qty#22, cr_item_qty#20] +Right output [2]: [item_id#31, wr_item_qty#32] +Arguments: [item_id#21], [item_id#31], Inner, BuildRight +(46) CometProject +Input [5]: [item_id#21, sr_item_qty#22, cr_item_qty#20, item_id#31, wr_item_qty#32] +Arguments: [item_id#21, sr_item_qty#22, sr_dev#33, cr_item_qty#20, cr_dev#34, wr_item_qty#32, wr_dev#35, average#36], [item_id#21, sr_item_qty#22, (((cast(sr_item_qty#22 as double) / cast(((sr_item_qty#22 + cr_item_qty#20) + wr_item_qty#32) as double)) / 3.0) * 100.0) AS sr_dev#33, cr_item_qty#20, (((cast(cr_item_qty#20 as double) / cast(((sr_item_qty#22 + cr_item_qty#20) + wr_item_qty#32) as double)) / 3.0) * 100.0) AS cr_dev#34, wr_item_qty#32, (((cast(wr_item_qty#32 as double) / cast(((sr_item_qty#22 + cr_item_qty#20) + wr_item_qty#32) as double)) / 3.0) * 100.0) AS wr_dev#35, (cast(((sr_item_qty#22 + cr_item_qty#20) + wr_item_qty#32) as decimal(20,0)) / 3.0) AS average#36] -(51) Scan parquet spark_catalog.default.date_dim +(47) CometTakeOrderedAndProject +Input [8]: [item_id#21, sr_item_qty#22, sr_dev#33, cr_item_qty#20, cr_dev#34, wr_item_qty#32, wr_dev#35, average#36] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_id#21 ASC NULLS FIRST,sr_item_qty#22 ASC NULLS FIRST], output=[item_id#21,sr_item_qty#22,sr_dev#33,cr_item_qty#20,cr_dev#34,wr_item_qty#32,wr_dev#35,average#36]), [item_id#21, sr_item_qty#22, sr_dev#33, cr_item_qty#20, cr_dev#34, wr_item_qty#32, wr_dev#35, average#36], 100, [item_id#21 ASC NULLS FIRST, sr_item_qty#22 ASC NULLS FIRST], [item_id#21, sr_item_qty#22, sr_dev#33, cr_item_qty#20, cr_dev#34, wr_item_qty#32, wr_dev#35, average#36] + +(48) ColumnarToRow [codegen id : 1] +Input [8]: [item_id#21, sr_item_qty#22, sr_dev#33, cr_item_qty#20, cr_dev#34, wr_item_qty#32, wr_dev#35, average#36] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 +BroadcastExchange (62) ++- * ColumnarToRow (61) + +- CometProject (60) + +- CometBroadcastHashJoin (59) + :- CometFilter (50) + : +- CometScan parquet spark_catalog.default.date_dim (49) + +- CometBroadcastExchange (58) + +- CometProject (57) + +- CometBroadcastHashJoin (56) + :- CometScan parquet spark_catalog.default.date_dim (51) + +- CometBroadcastExchange (55) + +- CometProject (54) + +- CometFilter (53) + +- CometScan parquet spark_catalog.default.date_dim (52) + + +(49) 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 -(52) CometFilter +(50) CometFilter Input [2]: [d_date_sk#7, d_date#8] Condition : isnotnull(d_date_sk#7) -(53) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date#9, d_week_seq#40] +(51) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date#9, d_week_seq#37] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] ReadSchema: struct -(54) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date#41, d_week_seq#42] +(52) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date#38, d_week_seq#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] ReadSchema: struct -(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) +(53) CometFilter +Input [2]: [d_date#38, d_week_seq#39] +Condition : cast(d_date#38 as string) IN (2000-06-30,2000-09-27,2000-11-17) -(56) CometProject -Input [2]: [d_date#41, d_week_seq#42] -Arguments: [d_week_seq#42], [d_week_seq#42] +(54) CometProject +Input [2]: [d_date#38, d_week_seq#39] +Arguments: [d_week_seq#39], [d_week_seq#39] -(57) CometBroadcastExchange -Input [1]: [d_week_seq#42] -Arguments: [d_week_seq#42] +(55) CometBroadcastExchange +Input [1]: [d_week_seq#39] +Arguments: [d_week_seq#39] -(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 +(56) CometBroadcastHashJoin +Left output [2]: [d_date#9, d_week_seq#37] +Right output [1]: [d_week_seq#39] +Arguments: [d_week_seq#37], [d_week_seq#39], LeftSemi, BuildRight -(59) CometProject -Input [2]: [d_date#9, d_week_seq#40] +(57) CometProject +Input [2]: [d_date#9, d_week_seq#37] Arguments: [d_date#9], [d_date#9] -(60) CometBroadcastExchange +(58) CometBroadcastExchange Input [1]: [d_date#9] Arguments: [d_date#9] -(61) CometBroadcastHashJoin +(59) 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 -(62) CometProject +(60) CometProject Input [2]: [d_date_sk#7, d_date#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(63) ColumnarToRow [codegen id : 1] +(61) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(64) BroadcastExchange +(62) 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] -Subquery:2 Hosting operator id = 20 Hosting Expression = cr_returned_date_sk#16 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 19 Hosting Expression = cr_returned_date_sk#13 IN dynamicpruning#4 -Subquery:3 Hosting operator id = 35 Hosting Expression = wr_returned_date_sk#27 IN dynamicpruning#4 +Subquery:3 Hosting operator id = 33 Hosting Expression = wr_returned_date_sk#25 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 390c20b7f..8365f11a4 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,83 +1,69 @@ -TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty,wr_dev,average] - 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] - BroadcastHashJoin [item_id,item_id] - HashAggregate [i_item_id,sum] [sum(sr_return_quantity),item_id,sr_item_qty,sum] - InputAdapter - Exchange [i_item_id] #1 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,sum,sr_return_quantity] - CometProject [sr_return_quantity,i_item_id] - CometBroadcastHashJoin [sr_return_quantity,sr_returned_date_sk,i_item_id,d_date_sk] - CometProject [sr_return_quantity,sr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [sr_item_sk,sr_return_quantity,sr_returned_date_sk,i_item_sk,i_item_id] - CometFilter [sr_item_sk,sr_return_quantity,sr_returned_date_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 (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] - CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] - CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] - CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [i_item_sk,i_item_id] #5 - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date] #3 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - HashAggregate [i_item_id,sum] [sum(cr_return_quantity),item_id,cr_item_qty,sum] - InputAdapter - Exchange [i_item_id] #8 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,sum,cr_return_quantity] - CometProject [cr_return_quantity,i_item_id] - CometBroadcastHashJoin [cr_return_quantity,cr_returned_date_sk,i_item_id,d_date_sk] - CometProject [cr_return_quantity,cr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [cr_item_sk,cr_return_quantity,cr_returned_date_sk,i_item_sk,i_item_id] - CometFilter [cr_item_sk,cr_return_quantity,cr_returned_date_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [i_item_sk,i_item_id] #5 - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (5) - HashAggregate [i_item_id,sum] [sum(wr_return_quantity),item_id,wr_item_qty,sum] - InputAdapter - Exchange [i_item_id] #10 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,sum,wr_return_quantity] - CometProject [wr_return_quantity,i_item_id] - CometBroadcastHashJoin [wr_return_quantity,wr_returned_date_sk,i_item_id,d_date_sk] - CometProject [wr_return_quantity,wr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [wr_item_sk,wr_return_quantity,wr_returned_date_sk,i_item_sk,i_item_id] - CometFilter [wr_item_sk,wr_return_quantity,wr_returned_date_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [i_item_sk,i_item_id] #5 - ReusedExchange [d_date_sk] #6 +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty,wr_dev,average] + CometProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty,wr_dev,average] + CometBroadcastHashJoin [item_id,sr_item_qty,cr_item_qty,item_id,wr_item_qty] + CometProject [item_id,sr_item_qty,cr_item_qty] + CometBroadcastHashJoin [item_id,sr_item_qty,item_id,cr_item_qty] + CometHashAggregate [item_id,sr_item_qty,i_item_id,sum,sum(sr_return_quantity)] + CometColumnarExchange [i_item_id] #1 + CometHashAggregate [i_item_id,sum,sr_return_quantity] + CometProject [sr_return_quantity,i_item_id] + CometBroadcastHashJoin [sr_return_quantity,sr_returned_date_sk,i_item_id,d_date_sk] + CometProject [sr_return_quantity,sr_returned_date_sk,i_item_id] + CometBroadcastHashJoin [sr_item_sk,sr_return_quantity,sr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [sr_item_sk,sr_return_quantity,sr_returned_date_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 (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date] #3 + CometProject [d_date] + CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometBroadcastExchange [d_week_seq] #4 + CometProject [d_week_seq] + CometFilter [d_date,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometBroadcastExchange [i_item_sk,i_item_id] #5 + CometFilter [i_item_sk,i_item_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + ReusedExchange [d_date] #3 + CometBroadcastExchange [item_id,cr_item_qty] #7 + CometHashAggregate [item_id,cr_item_qty,i_item_id,sum,sum(cr_return_quantity)] + CometColumnarExchange [i_item_id] #8 + CometHashAggregate [i_item_id,sum,cr_return_quantity] + CometProject [cr_return_quantity,i_item_id] + CometBroadcastHashJoin [cr_return_quantity,cr_returned_date_sk,i_item_id,d_date_sk] + CometProject [cr_return_quantity,cr_returned_date_sk,i_item_id] + CometBroadcastHashJoin [cr_item_sk,cr_return_quantity,cr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [cr_item_sk,cr_return_quantity,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [i_item_sk,i_item_id] #5 + ReusedExchange [d_date_sk] #6 + CometBroadcastExchange [item_id,wr_item_qty] #9 + CometHashAggregate [item_id,wr_item_qty,i_item_id,sum,sum(wr_return_quantity)] + CometColumnarExchange [i_item_id] #10 + CometHashAggregate [i_item_id,sum,wr_return_quantity] + CometProject [wr_return_quantity,i_item_id] + CometBroadcastHashJoin [wr_return_quantity,wr_returned_date_sk,i_item_id,d_date_sk] + CometProject [wr_return_quantity,wr_returned_date_sk,i_item_id] + CometBroadcastHashJoin [wr_item_sk,wr_return_quantity,wr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [wr_item_sk,wr_return_quantity,wr_returned_date_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + 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/q85/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt index cdd66620e..968c7402e 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,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (45) -+- * HashAggregate (44) - +- Exchange (43) - +- * ColumnarToRow (42) +* ColumnarToRow (45) ++- CometTakeOrderedAndProject (44) + +- CometHashAggregate (43) + +- CometColumnarExchange (42) +- CometHashAggregate (41) +- CometProject (40) +- CometBroadcastHashJoin (39) @@ -243,23 +243,21 @@ 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))] -(42) ColumnarToRow [codegen id : 1] +(42) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(44) HashAggregate [codegen id : 2] +(43) CometHashAggregate 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)#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] -(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] +(44) CometTakeOrderedAndProject +Input [4]: [substr(r_reason_desc, 1, 20)#38, avg(ws_quantity)#39, avg(wr_refunded_cash)#40, avg(wr_fee)#41] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(r_reason_desc, 1, 20)#38 ASC NULLS FIRST,avg(ws_quantity)#39 ASC NULLS FIRST,avg(wr_refunded_cash)#40 ASC NULLS FIRST,avg(wr_fee)#41 ASC NULLS FIRST], output=[substr(r_reason_desc, 1, 20)#38,avg(ws_quantity)#39,avg(wr_refunded_cash)#40,avg(wr_fee)#41]), [substr(r_reason_desc, 1, 20)#38, avg(ws_quantity)#39, avg(wr_refunded_cash)#40, avg(wr_fee)#41], 100, [substr(r_reason_desc, 1, 20)#38 ASC NULLS FIRST, avg(ws_quantity)#39 ASC NULLS FIRST, avg(wr_refunded_cash)#40 ASC NULLS FIRST, avg(wr_fee)#41 ASC NULLS FIRST], [substr(r_reason_desc, 1, 20)#38, avg(ws_quantity)#39, avg(wr_refunded_cash)#40, avg(wr_fee)#41] + +(45) ColumnarToRow [codegen id : 1] +Input [4]: [substr(r_reason_desc, 1, 20)#38, avg(ws_quantity)#39, avg(wr_refunded_cash)#40, avg(wr_fee)#41] ===== Subqueries ===== 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 2313467ae..9467b5c77 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,57 +1,55 @@ -TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee)] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [r_reason_desc,sum,count,sum,count,sum,count,ws_quantity,wr_refunded_cash,wr_fee] - CometProject [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] - CometBroadcastHashJoin [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash,r_reason_sk,r_reason_desc] - CometProject [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash,d_date_sk] - CometProject [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash,ca_address_sk,ca_state] - CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [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,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 [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,cd_demo_sk,cd_marital_status,cd_education_status] - 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,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,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_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,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] - CometBroadcastExchange [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] #2 - CometFilter [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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_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] - 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] - CometBroadcastExchange [wp_web_page_sk] #4 - CometFilter [wp_web_page_sk] - CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #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] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee)] + CometHashAggregate [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee),r_reason_desc,sum,count,sum,count,sum,count,avg(ws_quantity),avg(UnscaledValue(wr_refunded_cash)),avg(UnscaledValue(wr_fee))] + CometColumnarExchange [r_reason_desc] #1 + CometHashAggregate [r_reason_desc,sum,count,sum,count,sum,count,ws_quantity,wr_refunded_cash,wr_fee] + CometProject [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] + CometBroadcastHashJoin [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash,r_reason_sk,r_reason_desc] + CometProject [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash] + CometBroadcastHashJoin [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash,d_date_sk] + CometProject [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash] + CometBroadcastHashJoin [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash,ca_address_sk,ca_state] + CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash] + CometBroadcastHashJoin [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,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 [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,cd_demo_sk,cd_marital_status,cd_education_status] + 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,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,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_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,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] + CometBroadcastExchange [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] #2 + CometFilter [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + 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_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] + 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] + CometBroadcastExchange [wp_web_page_sk] #4 + CometFilter [wp_web_page_sk] + CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #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] - CometBroadcastExchange [ca_address_sk,ca_state] #7 - CometProject [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #8 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [r_reason_sk,r_reason_desc] #9 - CometFilter [r_reason_sk,r_reason_desc] - CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #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 [ca_address_sk,ca_state] #7 + CometProject [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + CometBroadcastExchange [d_date_sk] #8 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [r_reason_sk,r_reason_desc] #9 + CometFilter [r_reason_sk,r_reason_desc] + 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 900d23064..3d0fe32c7 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 @@ -2,11 +2,11 @@ TakeOrderedAndProject (23) +- * Project (22) +- Window (21) - +- * Sort (20) - +- Exchange (19) - +- * HashAggregate (18) - +- Exchange (17) - +- * ColumnarToRow (16) + +- * ColumnarToRow (20) + +- CometSort (19) + +- CometColumnarExchange (18) + +- CometHashAggregate (17) + +- CometColumnarExchange (16) +- CometHashAggregate (15) +- CometExpand (14) +- CometProject (13) @@ -97,39 +97,37 @@ 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))] -(16) ColumnarToRow [codegen id : 1] +(16) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(17) CometHashAggregate 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#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] -(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) CometColumnarExchange +Input [7]: [total_sum#14, i_category#10, i_class#11, lochierarchy#15, _w0#16, _w1#17, _w2#18] +Arguments: hashpartitioning(_w1#17, _w2#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(19) CometSort +Input [7]: [total_sum#14, i_category#10, i_class#11, lochierarchy#15, _w0#16, _w1#17, _w2#18] +Arguments: [total_sum#14, i_category#10, i_class#11, lochierarchy#15, _w0#16, _w1#17, _w2#18], [_w1#17 ASC NULLS FIRST, _w2#18 ASC NULLS FIRST, _w0#16 DESC NULLS LAST] -(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 +(20) ColumnarToRow [codegen id : 1] +Input [7]: [total_sum#14, i_category#10, i_class#11, lochierarchy#15, _w0#16, _w1#17, _w2#18] (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] +Input [7]: [total_sum#14, i_category#10, i_class#11, lochierarchy#15, _w0#16, _w1#17, _w2#18] +Arguments: [rank(_w0#16) windowspecdefinition(_w1#17, _w2#18, _w0#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#19], [_w1#17, _w2#18], [_w0#16 DESC NULLS LAST] -(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] +(22) Project [codegen id : 2] +Output [5]: [total_sum#14, i_category#10, i_class#11, lochierarchy#15, rank_within_parent#19] +Input [8]: [total_sum#14, i_category#10, i_class#11, lochierarchy#15, _w0#16, _w1#17, _w2#18, rank_within_parent#19] (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] +Input [5]: [total_sum#14, i_category#10, i_class#11, lochierarchy#15, rank_within_parent#19] +Arguments: 100, [lochierarchy#15 DESC NULLS LAST, CASE WHEN (lochierarchy#15 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#19 ASC NULLS FIRST], [total_sum#14, i_category#10, i_class#11, lochierarchy#15, rank_within_parent#19] ===== Subqueries ===== 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 a6d1c25fc..bbe4a2d2f 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,39 +1,35 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (4) + WholeStageCodegen (2) Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [_w0,_w1,_w2] - WholeStageCodegen (3) - Sort [_w1,_w2,_w0] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [_w1,_w2] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_category,i_class,spark_grouping_id,sum,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,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometFilter [i_item_sk,i_class,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] + CometColumnarExchange [_w1,_w2] #1 + CometHashAggregate [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2,spark_grouping_id,sum,sum(UnscaledValue(ws_net_paid))] + CometColumnarExchange [i_category,i_class,spark_grouping_id] #2 + CometHashAggregate [i_category,i_class,spark_grouping_id,sum,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,ws_net_paid,i_item_sk,i_class,i_category] + CometProject [ws_item_sk,ws_net_paid] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometFilter [i_item_sk,i_class,i_category] + 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 0434066e6..d7ea7098e 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,53 +1,55 @@ == Physical Plan == -* 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) +* HashAggregate (51) ++- * ColumnarToRow (50) + +- CometColumnarExchange (49) + +- RowToColumnar (48) + +- * HashAggregate (47) + +- * Project (46) + +- * BroadcastHashJoin LeftAnti BuildRight (45) + :- * BroadcastHashJoin LeftAnti BuildRight (31) + : :- * ColumnarToRow (17) + : : +- CometHashAggregate (16) + : : +- CometColumnarExchange (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) + : +- * ColumnarToRow (29) + : +- CometHashAggregate (28) + : +- CometColumnarExchange (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) + +- * ColumnarToRow (43) + +- CometHashAggregate (42) + +- CometColumnarExchange (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 @@ -119,19 +121,17 @@ 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: [] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(17) HashAggregate [codegen id : 6] +(16) 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#9, c_first_name#8, d_date#5] + +(17) ColumnarToRow [codegen id : 3] +Input [3]: [c_last_name#9, c_first_name#8, d_date#5] (18) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_bill_customer_sk#10, cs_sold_date_sk#11] @@ -174,25 +174,23 @@ 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: [] -(27) ColumnarToRow [codegen id : 2] +(27) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=2] -(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] +(28) 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#17, c_first_name#16, d_date#14] + +(29) ColumnarToRow [codegen id : 1] +Input [3]: [c_last_name#17, c_first_name#16, d_date#14] (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] -(31) BroadcastHashJoin [codegen id : 6] +(31) BroadcastHashJoin [codegen id : 3] 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 @@ -239,46 +237,50 @@ 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: [] -(41) ColumnarToRow [codegen id : 4] +(41) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=4] -(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] - -(43) HashAggregate [codegen id : 5] +(42) 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#25, c_first_name#24, d_date#22] + +(43) ColumnarToRow [codegen id : 2] +Input [3]: [c_last_name#25, c_first_name#24, d_date#22] (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] -(45) BroadcastHashJoin [codegen id : 6] +(45) BroadcastHashJoin [codegen id : 3] 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 -(46) Project [codegen id : 6] +(46) Project [codegen id : 3] Output: [] Input [3]: [c_last_name#9, c_first_name#8, d_date#5] -(47) HashAggregate [codegen id : 6] +(47) HashAggregate [codegen id : 3] Input: [] Keys: [] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#26] Results [1]: [count#27] -(48) Exchange +(48) RowToColumnar +Input [1]: [count#27] + +(49) CometColumnarExchange +Input [1]: [count#27] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(50) ColumnarToRow [codegen id : 4] Input [1]: [count#27] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(49) HashAggregate [codegen id : 7] +(51) HashAggregate [codegen id : 4] Input [1]: [count#27] Keys: [] Functions [1]: [count(1)] @@ -288,32 +290,32 @@ 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 (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 +(52) 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 -(51) CometFilter +(53) 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)) -(52) CometProject +(54) 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] -(53) ColumnarToRow [codegen id : 1] +(55) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#4, d_date#5] -(54) BroadcastExchange +(56) BroadcastExchange Input [2]: [d_date_sk#4, d_date#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] 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 9d667265c..69f33a57a 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,18 +1,18 @@ -WholeStageCodegen (7) +WholeStageCodegen (4) HashAggregate [count] [count(1),count(1),count] - InputAdapter - Exchange #1 - WholeStageCodegen (6) - HashAggregate [count,count] - Project - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometColumnarExchange #1 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [count,count] + Project + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + ColumnarToRow + InputAdapter + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #2 CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] @@ -35,15 +35,13 @@ WholeStageCodegen (7) CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #7 CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] CometBroadcastHashJoin [cs_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] @@ -54,15 +52,13 @@ WholeStageCodegen (7) ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #9 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #9 CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] CometBroadcastHashJoin [ws_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] 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 3f905ebe9..cd9e4817c 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 @@ -6,9 +6,9 @@ : : : :- * BroadcastNestedLoopJoin Inner BuildRight (88) : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (67) : : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (46) -: : : : : : :- * HashAggregate (25) -: : : : : : : +- Exchange (24) -: : : : : : : +- * ColumnarToRow (23) +: : : : : : :- * ColumnarToRow (25) +: : : : : : : +- CometHashAggregate (24) +: : : : : : : +- CometColumnarExchange (23) : : : : : : : +- CometHashAggregate (22) : : : : : : : +- CometProject (21) : : : : : : : +- CometBroadcastHashJoin (20) @@ -32,9 +32,9 @@ : : : : : : : +- CometFilter (17) : : : : : : : +- CometScan parquet spark_catalog.default.store (16) : : : : : : +- BroadcastExchange (45) -: : : : : : +- * HashAggregate (44) -: : : : : : +- Exchange (43) -: : : : : : +- * ColumnarToRow (42) +: : : : : : +- * ColumnarToRow (44) +: : : : : : +- CometHashAggregate (43) +: : : : : : +- CometColumnarExchange (42) : : : : : : +- CometHashAggregate (41) : : : : : : +- CometProject (40) : : : : : : +- CometBroadcastHashJoin (39) @@ -52,9 +52,9 @@ : : : : : : : +- CometScan parquet spark_catalog.default.time_dim (32) : : : : : : +- ReusedExchange (38) : : : : : +- BroadcastExchange (66) -: : : : : +- * HashAggregate (65) -: : : : : +- Exchange (64) -: : : : : +- * ColumnarToRow (63) +: : : : : +- * ColumnarToRow (65) +: : : : : +- CometHashAggregate (64) +: : : : : +- CometColumnarExchange (63) : : : : : +- CometHashAggregate (62) : : : : : +- CometProject (61) : : : : : +- CometBroadcastHashJoin (60) @@ -72,9 +72,9 @@ : : : : : : +- CometScan parquet spark_catalog.default.time_dim (53) : : : : : +- ReusedExchange (59) : : : : +- BroadcastExchange (87) -: : : : +- * HashAggregate (86) -: : : : +- Exchange (85) -: : : : +- * ColumnarToRow (84) +: : : : +- * ColumnarToRow (86) +: : : : +- CometHashAggregate (85) +: : : : +- CometColumnarExchange (84) : : : : +- CometHashAggregate (83) : : : : +- CometProject (82) : : : : +- CometBroadcastHashJoin (81) @@ -92,9 +92,9 @@ : : : : : +- CometScan parquet spark_catalog.default.time_dim (74) : : : : +- ReusedExchange (80) : : : +- BroadcastExchange (108) -: : : +- * HashAggregate (107) -: : : +- Exchange (106) -: : : +- * ColumnarToRow (105) +: : : +- * ColumnarToRow (107) +: : : +- CometHashAggregate (106) +: : : +- CometColumnarExchange (105) : : : +- CometHashAggregate (104) : : : +- CometProject (103) : : : +- CometBroadcastHashJoin (102) @@ -112,9 +112,9 @@ : : : : +- CometScan parquet spark_catalog.default.time_dim (95) : : : +- ReusedExchange (101) : : +- BroadcastExchange (129) -: : +- * HashAggregate (128) -: : +- Exchange (127) -: : +- * ColumnarToRow (126) +: : +- * ColumnarToRow (128) +: : +- CometHashAggregate (127) +: : +- CometColumnarExchange (126) : : +- CometHashAggregate (125) : : +- CometProject (124) : : +- CometBroadcastHashJoin (123) @@ -132,9 +132,9 @@ : : : +- CometScan parquet spark_catalog.default.time_dim (116) : : +- ReusedExchange (122) : +- BroadcastExchange (150) -: +- * HashAggregate (149) -: +- Exchange (148) -: +- * ColumnarToRow (147) +: +- * ColumnarToRow (149) +: +- CometHashAggregate (148) +: +- CometColumnarExchange (147) : +- CometHashAggregate (146) : +- CometProject (145) : +- CometBroadcastHashJoin (144) @@ -152,9 +152,9 @@ : : +- CometScan parquet spark_catalog.default.time_dim (137) : +- ReusedExchange (143) +- BroadcastExchange (171) - +- * HashAggregate (170) - +- Exchange (169) - +- * ColumnarToRow (168) + +- * ColumnarToRow (170) + +- CometHashAggregate (169) + +- CometColumnarExchange (168) +- CometHashAggregate (167) +- CometProject (166) +- CometBroadcastHashJoin (165) @@ -276,668 +276,652 @@ Input: [] Keys: [] Functions [1]: [partial_count(1)] -(23) ColumnarToRow [codegen id : 1] +(23) CometColumnarExchange Input [1]: [count#13] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(24) Exchange -Input [1]: [count#13] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] - -(25) HashAggregate [codegen id : 16] +(24) CometHashAggregate Input [1]: [count#13] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#14] -Results [1]: [count(1)#14 AS h8_30_to_9#15] + +(25) ColumnarToRow [codegen id : 8] +Input [1]: [h8_30_to_9#14] (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] +Output [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] 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 (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)) +Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] +Condition : ((isnotnull(ss_hdemo_sk#16) AND isnotnull(ss_sold_time_sk#15)) AND isnotnull(ss_store_sk#17)) (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] +Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] +Arguments: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17], [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17] (29) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#20] +Output [1]: [hd_demo_sk#19] (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 +Left output [3]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17] +Right output [1]: [hd_demo_sk#19] +Arguments: [ss_hdemo_sk#16], [hd_demo_sk#19], Inner, BuildRight (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] +Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, hd_demo_sk#19] +Arguments: [ss_sold_time_sk#15, ss_store_sk#17], [ss_sold_time_sk#15, ss_store_sk#17] (32) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#21, t_hour#22, t_minute#23] +Output [3]: [t_time_sk#20, t_hour#21, t_minute#22] 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 (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)) +Input [3]: [t_time_sk#20, t_hour#21, t_minute#22] +Condition : ((((isnotnull(t_hour#21) AND isnotnull(t_minute#22)) AND (t_hour#21 = 9)) AND (t_minute#22 < 30)) AND isnotnull(t_time_sk#20)) (34) CometProject -Input [3]: [t_time_sk#21, t_hour#22, t_minute#23] -Arguments: [t_time_sk#21], [t_time_sk#21] +Input [3]: [t_time_sk#20, t_hour#21, t_minute#22] +Arguments: [t_time_sk#20], [t_time_sk#20] (35) CometBroadcastExchange -Input [1]: [t_time_sk#21] -Arguments: [t_time_sk#21] +Input [1]: [t_time_sk#20] +Arguments: [t_time_sk#20] (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 +Left output [2]: [ss_sold_time_sk#15, ss_store_sk#17] +Right output [1]: [t_time_sk#20] +Arguments: [ss_sold_time_sk#15], [t_time_sk#20], Inner, BuildRight (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] +Input [3]: [ss_sold_time_sk#15, ss_store_sk#17, t_time_sk#20] +Arguments: [ss_store_sk#17], [ss_store_sk#17] (38) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#24] +Output [1]: [s_store_sk#23] (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 +Left output [1]: [ss_store_sk#17] +Right output [1]: [s_store_sk#23] +Arguments: [ss_store_sk#17], [s_store_sk#23], Inner, BuildRight (40) CometProject -Input [2]: [ss_store_sk#18, s_store_sk#24] +Input [2]: [ss_store_sk#17, s_store_sk#23] (41) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -(42) ColumnarToRow [codegen id : 2] -Input [1]: [count#25] +(42) CometColumnarExchange +Input [1]: [count#24] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(43) Exchange -Input [1]: [count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] - -(44) HashAggregate [codegen id : 3] -Input [1]: [count#25] +(43) CometHashAggregate +Input [1]: [count#24] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#26] -Results [1]: [count(1)#26 AS h9_to_9_30#27] + +(44) ColumnarToRow [codegen id : 1] +Input [1]: [h9_to_9_30#25] (45) BroadcastExchange -Input [1]: [h9_to_9_30#27] +Input [1]: [h9_to_9_30#25] Arguments: IdentityBroadcastMode, [plan_id=3] -(46) BroadcastNestedLoopJoin [codegen id : 16] +(46) BroadcastNestedLoopJoin [codegen id : 8] Join type: Inner Join condition: None (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] +Output [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] 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 (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)) +Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] +Condition : ((isnotnull(ss_hdemo_sk#27) AND isnotnull(ss_sold_time_sk#26)) AND isnotnull(ss_store_sk#28)) (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] +Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] +Arguments: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28], [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28] (50) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#32] +Output [1]: [hd_demo_sk#30] (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 +Left output [3]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28] +Right output [1]: [hd_demo_sk#30] +Arguments: [ss_hdemo_sk#27], [hd_demo_sk#30], Inner, BuildRight (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] +Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, hd_demo_sk#30] +Arguments: [ss_sold_time_sk#26, ss_store_sk#28], [ss_sold_time_sk#26, ss_store_sk#28] (53) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#33, t_hour#34, t_minute#35] +Output [3]: [t_time_sk#31, t_hour#32, t_minute#33] 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 (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)) +Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] +Condition : ((((isnotnull(t_hour#32) AND isnotnull(t_minute#33)) AND (t_hour#32 = 9)) AND (t_minute#33 >= 30)) AND isnotnull(t_time_sk#31)) (55) CometProject -Input [3]: [t_time_sk#33, t_hour#34, t_minute#35] -Arguments: [t_time_sk#33], [t_time_sk#33] +Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] +Arguments: [t_time_sk#31], [t_time_sk#31] (56) CometBroadcastExchange -Input [1]: [t_time_sk#33] -Arguments: [t_time_sk#33] +Input [1]: [t_time_sk#31] +Arguments: [t_time_sk#31] (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 +Left output [2]: [ss_sold_time_sk#26, ss_store_sk#28] +Right output [1]: [t_time_sk#31] +Arguments: [ss_sold_time_sk#26], [t_time_sk#31], Inner, BuildRight (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] +Input [3]: [ss_sold_time_sk#26, ss_store_sk#28, t_time_sk#31] +Arguments: [ss_store_sk#28], [ss_store_sk#28] (59) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#36] +Output [1]: [s_store_sk#34] (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 +Left output [1]: [ss_store_sk#28] +Right output [1]: [s_store_sk#34] +Arguments: [ss_store_sk#28], [s_store_sk#34], Inner, BuildRight (61) CometProject -Input [2]: [ss_store_sk#30, s_store_sk#36] +Input [2]: [ss_store_sk#28, s_store_sk#34] (62) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -(63) ColumnarToRow [codegen id : 4] -Input [1]: [count#37] +(63) CometColumnarExchange +Input [1]: [count#35] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(64) Exchange -Input [1]: [count#37] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] - -(65) HashAggregate [codegen id : 5] -Input [1]: [count#37] +(64) CometHashAggregate +Input [1]: [count#35] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#38] -Results [1]: [count(1)#38 AS h9_30_to_10#39] + +(65) ColumnarToRow [codegen id : 2] +Input [1]: [h9_30_to_10#36] (66) BroadcastExchange -Input [1]: [h9_30_to_10#39] +Input [1]: [h9_30_to_10#36] Arguments: IdentityBroadcastMode, [plan_id=5] -(67) BroadcastNestedLoopJoin [codegen id : 16] +(67) BroadcastNestedLoopJoin [codegen id : 8] Join type: Inner Join condition: None (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] +Output [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] 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 (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)) +Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] +Condition : ((isnotnull(ss_hdemo_sk#38) AND isnotnull(ss_sold_time_sk#37)) AND isnotnull(ss_store_sk#39)) (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] +Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] +Arguments: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39], [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39] (71) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#44] +Output [1]: [hd_demo_sk#41] (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 +Left output [3]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39] +Right output [1]: [hd_demo_sk#41] +Arguments: [ss_hdemo_sk#38], [hd_demo_sk#41], Inner, BuildRight (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] +Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, hd_demo_sk#41] +Arguments: [ss_sold_time_sk#37, ss_store_sk#39], [ss_sold_time_sk#37, ss_store_sk#39] (74) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#45, t_hour#46, t_minute#47] +Output [3]: [t_time_sk#42, t_hour#43, t_minute#44] 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 (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)) +Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] +Condition : ((((isnotnull(t_hour#43) AND isnotnull(t_minute#44)) AND (t_hour#43 = 10)) AND (t_minute#44 < 30)) AND isnotnull(t_time_sk#42)) (76) CometProject -Input [3]: [t_time_sk#45, t_hour#46, t_minute#47] -Arguments: [t_time_sk#45], [t_time_sk#45] +Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] +Arguments: [t_time_sk#42], [t_time_sk#42] (77) CometBroadcastExchange -Input [1]: [t_time_sk#45] -Arguments: [t_time_sk#45] +Input [1]: [t_time_sk#42] +Arguments: [t_time_sk#42] (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 +Left output [2]: [ss_sold_time_sk#37, ss_store_sk#39] +Right output [1]: [t_time_sk#42] +Arguments: [ss_sold_time_sk#37], [t_time_sk#42], Inner, BuildRight (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] +Input [3]: [ss_sold_time_sk#37, ss_store_sk#39, t_time_sk#42] +Arguments: [ss_store_sk#39], [ss_store_sk#39] (80) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#48] +Output [1]: [s_store_sk#45] (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 +Left output [1]: [ss_store_sk#39] +Right output [1]: [s_store_sk#45] +Arguments: [ss_store_sk#39], [s_store_sk#45], Inner, BuildRight (82) CometProject -Input [2]: [ss_store_sk#42, s_store_sk#48] +Input [2]: [ss_store_sk#39, s_store_sk#45] (83) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -(84) ColumnarToRow [codegen id : 6] -Input [1]: [count#49] +(84) CometColumnarExchange +Input [1]: [count#46] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(85) Exchange -Input [1]: [count#49] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] - -(86) HashAggregate [codegen id : 7] -Input [1]: [count#49] +(85) CometHashAggregate +Input [1]: [count#46] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#50] -Results [1]: [count(1)#50 AS h10_to_10_30#51] + +(86) ColumnarToRow [codegen id : 3] +Input [1]: [h10_to_10_30#47] (87) BroadcastExchange -Input [1]: [h10_to_10_30#51] +Input [1]: [h10_to_10_30#47] Arguments: IdentityBroadcastMode, [plan_id=7] -(88) BroadcastNestedLoopJoin [codegen id : 16] +(88) BroadcastNestedLoopJoin [codegen id : 8] Join type: Inner Join condition: None (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] +Output [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] 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 (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)) +Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] +Condition : ((isnotnull(ss_hdemo_sk#49) AND isnotnull(ss_sold_time_sk#48)) AND isnotnull(ss_store_sk#50)) (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] +Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] +Arguments: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50], [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50] (92) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#56] +Output [1]: [hd_demo_sk#52] (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 +Left output [3]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50] +Right output [1]: [hd_demo_sk#52] +Arguments: [ss_hdemo_sk#49], [hd_demo_sk#52], Inner, BuildRight (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] +Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, hd_demo_sk#52] +Arguments: [ss_sold_time_sk#48, ss_store_sk#50], [ss_sold_time_sk#48, ss_store_sk#50] (95) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#57, t_hour#58, t_minute#59] +Output [3]: [t_time_sk#53, t_hour#54, t_minute#55] 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 (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)) +Input [3]: [t_time_sk#53, t_hour#54, t_minute#55] +Condition : ((((isnotnull(t_hour#54) AND isnotnull(t_minute#55)) AND (t_hour#54 = 10)) AND (t_minute#55 >= 30)) AND isnotnull(t_time_sk#53)) (97) CometProject -Input [3]: [t_time_sk#57, t_hour#58, t_minute#59] -Arguments: [t_time_sk#57], [t_time_sk#57] +Input [3]: [t_time_sk#53, t_hour#54, t_minute#55] +Arguments: [t_time_sk#53], [t_time_sk#53] (98) CometBroadcastExchange -Input [1]: [t_time_sk#57] -Arguments: [t_time_sk#57] +Input [1]: [t_time_sk#53] +Arguments: [t_time_sk#53] (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 +Left output [2]: [ss_sold_time_sk#48, ss_store_sk#50] +Right output [1]: [t_time_sk#53] +Arguments: [ss_sold_time_sk#48], [t_time_sk#53], Inner, BuildRight (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] +Input [3]: [ss_sold_time_sk#48, ss_store_sk#50, t_time_sk#53] +Arguments: [ss_store_sk#50], [ss_store_sk#50] (101) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#60] +Output [1]: [s_store_sk#56] (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 +Left output [1]: [ss_store_sk#50] +Right output [1]: [s_store_sk#56] +Arguments: [ss_store_sk#50], [s_store_sk#56], Inner, BuildRight (103) CometProject -Input [2]: [ss_store_sk#54, s_store_sk#60] +Input [2]: [ss_store_sk#50, s_store_sk#56] (104) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -(105) ColumnarToRow [codegen id : 8] -Input [1]: [count#61] +(105) CometColumnarExchange +Input [1]: [count#57] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(106) Exchange -Input [1]: [count#61] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=8] - -(107) HashAggregate [codegen id : 9] -Input [1]: [count#61] +(106) CometHashAggregate +Input [1]: [count#57] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#62] -Results [1]: [count(1)#62 AS h10_30_to_11#63] + +(107) ColumnarToRow [codegen id : 4] +Input [1]: [h10_30_to_11#58] (108) BroadcastExchange -Input [1]: [h10_30_to_11#63] +Input [1]: [h10_30_to_11#58] Arguments: IdentityBroadcastMode, [plan_id=9] -(109) BroadcastNestedLoopJoin [codegen id : 16] +(109) BroadcastNestedLoopJoin [codegen id : 8] Join type: Inner Join condition: None (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] +Output [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] 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 (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)) +Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] +Condition : ((isnotnull(ss_hdemo_sk#60) AND isnotnull(ss_sold_time_sk#59)) AND isnotnull(ss_store_sk#61)) (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] +Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] +Arguments: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61], [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61] (113) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#68] +Output [1]: [hd_demo_sk#63] (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 +Left output [3]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61] +Right output [1]: [hd_demo_sk#63] +Arguments: [ss_hdemo_sk#60], [hd_demo_sk#63], Inner, BuildRight (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] +Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, hd_demo_sk#63] +Arguments: [ss_sold_time_sk#59, ss_store_sk#61], [ss_sold_time_sk#59, ss_store_sk#61] (116) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#69, t_hour#70, t_minute#71] +Output [3]: [t_time_sk#64, t_hour#65, t_minute#66] 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 (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)) +Input [3]: [t_time_sk#64, t_hour#65, t_minute#66] +Condition : ((((isnotnull(t_hour#65) AND isnotnull(t_minute#66)) AND (t_hour#65 = 11)) AND (t_minute#66 < 30)) AND isnotnull(t_time_sk#64)) (118) CometProject -Input [3]: [t_time_sk#69, t_hour#70, t_minute#71] -Arguments: [t_time_sk#69], [t_time_sk#69] +Input [3]: [t_time_sk#64, t_hour#65, t_minute#66] +Arguments: [t_time_sk#64], [t_time_sk#64] (119) CometBroadcastExchange -Input [1]: [t_time_sk#69] -Arguments: [t_time_sk#69] +Input [1]: [t_time_sk#64] +Arguments: [t_time_sk#64] (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 +Left output [2]: [ss_sold_time_sk#59, ss_store_sk#61] +Right output [1]: [t_time_sk#64] +Arguments: [ss_sold_time_sk#59], [t_time_sk#64], Inner, BuildRight (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] +Input [3]: [ss_sold_time_sk#59, ss_store_sk#61, t_time_sk#64] +Arguments: [ss_store_sk#61], [ss_store_sk#61] (122) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#72] +Output [1]: [s_store_sk#67] (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 +Left output [1]: [ss_store_sk#61] +Right output [1]: [s_store_sk#67] +Arguments: [ss_store_sk#61], [s_store_sk#67], Inner, BuildRight (124) CometProject -Input [2]: [ss_store_sk#66, s_store_sk#72] +Input [2]: [ss_store_sk#61, s_store_sk#67] (125) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -(126) ColumnarToRow [codegen id : 10] -Input [1]: [count#73] +(126) CometColumnarExchange +Input [1]: [count#68] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(127) Exchange -Input [1]: [count#73] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] - -(128) HashAggregate [codegen id : 11] -Input [1]: [count#73] +(127) CometHashAggregate +Input [1]: [count#68] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#74] -Results [1]: [count(1)#74 AS h11_to_11_30#75] + +(128) ColumnarToRow [codegen id : 5] +Input [1]: [h11_to_11_30#69] (129) BroadcastExchange -Input [1]: [h11_to_11_30#75] +Input [1]: [h11_to_11_30#69] Arguments: IdentityBroadcastMode, [plan_id=11] -(130) BroadcastNestedLoopJoin [codegen id : 16] +(130) BroadcastNestedLoopJoin [codegen id : 8] Join type: Inner Join condition: None (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] +Output [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] 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 (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)) +Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] +Condition : ((isnotnull(ss_hdemo_sk#71) AND isnotnull(ss_sold_time_sk#70)) AND isnotnull(ss_store_sk#72)) (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] +Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] +Arguments: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72], [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72] (134) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#80] +Output [1]: [hd_demo_sk#74] (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 +Left output [3]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72] +Right output [1]: [hd_demo_sk#74] +Arguments: [ss_hdemo_sk#71], [hd_demo_sk#74], Inner, BuildRight (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] +Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, hd_demo_sk#74] +Arguments: [ss_sold_time_sk#70, ss_store_sk#72], [ss_sold_time_sk#70, ss_store_sk#72] (137) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#81, t_hour#82, t_minute#83] +Output [3]: [t_time_sk#75, t_hour#76, t_minute#77] 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 (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)) +Input [3]: [t_time_sk#75, t_hour#76, t_minute#77] +Condition : ((((isnotnull(t_hour#76) AND isnotnull(t_minute#77)) AND (t_hour#76 = 11)) AND (t_minute#77 >= 30)) AND isnotnull(t_time_sk#75)) (139) CometProject -Input [3]: [t_time_sk#81, t_hour#82, t_minute#83] -Arguments: [t_time_sk#81], [t_time_sk#81] +Input [3]: [t_time_sk#75, t_hour#76, t_minute#77] +Arguments: [t_time_sk#75], [t_time_sk#75] (140) CometBroadcastExchange -Input [1]: [t_time_sk#81] -Arguments: [t_time_sk#81] +Input [1]: [t_time_sk#75] +Arguments: [t_time_sk#75] (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 +Left output [2]: [ss_sold_time_sk#70, ss_store_sk#72] +Right output [1]: [t_time_sk#75] +Arguments: [ss_sold_time_sk#70], [t_time_sk#75], Inner, BuildRight (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] +Input [3]: [ss_sold_time_sk#70, ss_store_sk#72, t_time_sk#75] +Arguments: [ss_store_sk#72], [ss_store_sk#72] (143) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#84] +Output [1]: [s_store_sk#78] (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 +Left output [1]: [ss_store_sk#72] +Right output [1]: [s_store_sk#78] +Arguments: [ss_store_sk#72], [s_store_sk#78], Inner, BuildRight (145) CometProject -Input [2]: [ss_store_sk#78, s_store_sk#84] +Input [2]: [ss_store_sk#72, s_store_sk#78] (146) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -(147) ColumnarToRow [codegen id : 12] -Input [1]: [count#85] +(147) CometColumnarExchange +Input [1]: [count#79] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(148) Exchange -Input [1]: [count#85] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] - -(149) HashAggregate [codegen id : 13] -Input [1]: [count#85] +(148) CometHashAggregate +Input [1]: [count#79] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#86] -Results [1]: [count(1)#86 AS h11_30_to_12#87] + +(149) ColumnarToRow [codegen id : 6] +Input [1]: [h11_30_to_12#80] (150) BroadcastExchange -Input [1]: [h11_30_to_12#87] +Input [1]: [h11_30_to_12#80] Arguments: IdentityBroadcastMode, [plan_id=13] -(151) BroadcastNestedLoopJoin [codegen id : 16] +(151) BroadcastNestedLoopJoin [codegen id : 8] Join type: Inner Join condition: None (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] +Output [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] 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 (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)) +Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] +Condition : ((isnotnull(ss_hdemo_sk#82) AND isnotnull(ss_sold_time_sk#81)) AND isnotnull(ss_store_sk#83)) (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] +Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] +Arguments: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83], [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83] (155) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#92] +Output [1]: [hd_demo_sk#85] (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 +Left output [3]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83] +Right output [1]: [hd_demo_sk#85] +Arguments: [ss_hdemo_sk#82], [hd_demo_sk#85], Inner, BuildRight (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] +Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, hd_demo_sk#85] +Arguments: [ss_sold_time_sk#81, ss_store_sk#83], [ss_sold_time_sk#81, ss_store_sk#83] (158) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#93, t_hour#94, t_minute#95] +Output [3]: [t_time_sk#86, t_hour#87, t_minute#88] 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 (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)) +Input [3]: [t_time_sk#86, t_hour#87, t_minute#88] +Condition : ((((isnotnull(t_hour#87) AND isnotnull(t_minute#88)) AND (t_hour#87 = 12)) AND (t_minute#88 < 30)) AND isnotnull(t_time_sk#86)) (160) CometProject -Input [3]: [t_time_sk#93, t_hour#94, t_minute#95] -Arguments: [t_time_sk#93], [t_time_sk#93] +Input [3]: [t_time_sk#86, t_hour#87, t_minute#88] +Arguments: [t_time_sk#86], [t_time_sk#86] (161) CometBroadcastExchange -Input [1]: [t_time_sk#93] -Arguments: [t_time_sk#93] +Input [1]: [t_time_sk#86] +Arguments: [t_time_sk#86] (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 +Left output [2]: [ss_sold_time_sk#81, ss_store_sk#83] +Right output [1]: [t_time_sk#86] +Arguments: [ss_sold_time_sk#81], [t_time_sk#86], Inner, BuildRight (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] +Input [3]: [ss_sold_time_sk#81, ss_store_sk#83, t_time_sk#86] +Arguments: [ss_store_sk#83], [ss_store_sk#83] (164) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#96] +Output [1]: [s_store_sk#89] (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 +Left output [1]: [ss_store_sk#83] +Right output [1]: [s_store_sk#89] +Arguments: [ss_store_sk#83], [s_store_sk#89], Inner, BuildRight (166) CometProject -Input [2]: [ss_store_sk#90, s_store_sk#96] +Input [2]: [ss_store_sk#83, s_store_sk#89] (167) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -(168) ColumnarToRow [codegen id : 14] -Input [1]: [count#97] +(168) CometColumnarExchange +Input [1]: [count#90] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] -(169) Exchange -Input [1]: [count#97] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=14] - -(170) HashAggregate [codegen id : 15] -Input [1]: [count#97] +(169) CometHashAggregate +Input [1]: [count#90] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#98] -Results [1]: [count(1)#98 AS h12_to_12_30#99] + +(170) ColumnarToRow [codegen id : 7] +Input [1]: [h12_to_12_30#91] (171) BroadcastExchange -Input [1]: [h12_to_12_30#99] +Input [1]: [h12_to_12_30#91] Arguments: IdentityBroadcastMode, [plan_id=15] -(172) BroadcastNestedLoopJoin [codegen id : 16] +(172) BroadcastNestedLoopJoin [codegen id : 8] 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 d3af10d19..b01685ed0 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 (16) +WholeStageCodegen (8) BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin @@ -6,93 +6,39 @@ WholeStageCodegen (16) BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin - HashAggregate [count] [count(1),h8_30_to_9,count] + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] - CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_store_name] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] + CometHashAggregate [h8_30_to_9,count,count(1)] + CometColumnarExchange #1 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometBroadcastExchange [hd_demo_sk] #2 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange [t_time_sk] #3 + CometProject [t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + CometBroadcastExchange [s_store_sk] #4 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_store_name] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - HashAggregate [count] [count(1),h9_to_9_30,count] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange #6 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] - CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #7 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - HashAggregate [count] [count(1),h9_30_to_10,count] - InputAdapter - Exchange #9 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] - CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #10 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (7) - HashAggregate [count] [count(1),h10_to_10_30,count] - InputAdapter - Exchange #12 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter + CometHashAggregate [h9_to_9_30,count,count(1)] + CometColumnarExchange #6 CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] @@ -104,20 +50,18 @@ WholeStageCodegen (16) CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #13 + CometBroadcastExchange [t_time_sk] #7 CometProject [t_time_sk] CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (9) - HashAggregate [count] [count(1),h10_30_to_11,count] - InputAdapter - Exchange #15 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometHashAggregate [h9_30_to_10,count,count(1)] + CometColumnarExchange #9 CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] @@ -129,20 +73,18 @@ WholeStageCodegen (16) CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #16 + CometBroadcastExchange [t_time_sk] #10 CometProject [t_time_sk] CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (11) - HashAggregate [count] [count(1),h11_to_11_30,count] - InputAdapter - Exchange #18 - WholeStageCodegen (10) - ColumnarToRow - InputAdapter + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometHashAggregate [h10_to_10_30,count,count(1)] + CometColumnarExchange #12 CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] @@ -154,20 +96,18 @@ WholeStageCodegen (16) CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #19 + CometBroadcastExchange [t_time_sk] #13 CometProject [t_time_sk] CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #20 - WholeStageCodegen (13) - HashAggregate [count] [count(1),h11_30_to_12,count] - InputAdapter - Exchange #21 - WholeStageCodegen (12) - ColumnarToRow - InputAdapter + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometHashAggregate [h10_30_to_11,count,count(1)] + CometColumnarExchange #15 CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] @@ -179,20 +119,18 @@ WholeStageCodegen (16) CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #22 + CometBroadcastExchange [t_time_sk] #16 CometProject [t_time_sk] CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #23 - WholeStageCodegen (15) - HashAggregate [count] [count(1),h12_to_12_30,count] - InputAdapter - Exchange #24 - WholeStageCodegen (14) - ColumnarToRow - InputAdapter + InputAdapter + BroadcastExchange #17 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometHashAggregate [h11_to_11_30,count,count(1)] + CometColumnarExchange #18 CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] @@ -204,8 +142,54 @@ WholeStageCodegen (16) CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #25 + CometBroadcastExchange [t_time_sk] #19 CometProject [t_time_sk] CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 + InputAdapter + BroadcastExchange #20 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometHashAggregate [h11_30_to_12,count,count(1)] + CometColumnarExchange #21 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange [t_time_sk] #22 + CometProject [t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + ReusedExchange [s_store_sk] #4 + InputAdapter + BroadcastExchange #23 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometHashAggregate [h12_to_12_30,count,count(1)] + CometColumnarExchange #24 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange [t_time_sk] #25 + CometProject [t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] + 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 49dab66f0..6f973cee6 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 @@ -3,11 +3,11 @@ TakeOrderedAndProject (28) +- * Project (27) +- * Filter (26) +- Window (25) - +- * Sort (24) - +- Exchange (23) - +- * HashAggregate (22) - +- Exchange (21) - +- * ColumnarToRow (20) + +- * ColumnarToRow (24) + +- CometSort (23) + +- CometColumnarExchange (22) + +- CometHashAggregate (21) + +- CometColumnarExchange (20) +- CometHashAggregate (19) +- CometProject (18) +- CometBroadcastHashJoin (17) @@ -122,43 +122,41 @@ Input [7]: [i_brand#2, i_class#3, i_category#4, ss_sales_price#7, d_moy#12, s_st 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))] -(20) ColumnarToRow [codegen id : 1] +(20) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(22) HashAggregate [codegen id : 2] +(21) CometHashAggregate 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#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] -(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] +(22) CometColumnarExchange +Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#17, _w0#18] +Arguments: hashpartitioning(i_category#4, i_brand#2, s_store_name#14, s_company_name#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(23) CometSort +Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#17, _w0#18] +Arguments: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#17, _w0#18], [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] -(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) ColumnarToRow [codegen id : 1] +Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#17, _w0#18] (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] +Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#17, _w0#18] +Arguments: [avg(_w0#18) windowspecdefinition(i_category#4, i_brand#2, s_store_name#14, s_company_name#15, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#19], [i_category#4, i_brand#2, s_store_name#14, s_company_name#15] -(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) Filter [codegen id : 2] +Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#17, _w0#18, avg_monthly_sales#19] +Condition : CASE WHEN NOT (avg_monthly_sales#19 = 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#19)) / avg_monthly_sales#19) > 0.1000000000000000) END -(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) Project [codegen id : 2] +Output [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#17, avg_monthly_sales#19] +Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#17, _w0#18, avg_monthly_sales#19] (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] +Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#17, avg_monthly_sales#19] +Arguments: 100, [(sum_sales#17 - avg_monthly_sales#19) 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#17, avg_monthly_sales#19] ===== Subqueries ===== 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 aa858c158..26c510f5a 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,44 +1,40 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_class,i_brand,s_company_name,d_moy] - WholeStageCodegen (4) + WholeStageCodegen (2) 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 (3) - Sort [i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum,ss_sales_price] - CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] - CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometFilter [i_item_sk,i_brand,i_class,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #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] + CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 + CometHashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,sum,sum(UnscaledValue(ss_sales_price))] + CometColumnarExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 + CometHashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum,ss_sales_price] + CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] + CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_class,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_moy] #5 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt index 1e97f8a15..375b036fc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt @@ -26,10 +26,10 @@ Input [1]: [r_reason_sk#1] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#2, [id=#3] -* Project (12) -+- * HashAggregate (11) - +- Exchange (10) - +- * ColumnarToRow (9) +* ColumnarToRow (12) ++- CometProject (11) + +- CometHashAggregate (10) + +- CometColumnarExchange (9) +- CometHashAggregate (8) +- CometProject (7) +- CometFilter (6) @@ -56,33 +56,31 @@ Input [2]: [ss_ext_discount_amt#18, ss_net_paid#19] Keys: [] Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#18)), partial_avg(UnscaledValue(ss_net_paid#19))] -(9) ColumnarToRow [codegen id : 1] +(9) CometColumnarExchange Input [5]: [count#21, sum#22, count#23, sum#24, count#25] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(10) Exchange -Input [5]: [count#21, sum#22, count#23, sum#24, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] - -(11) HashAggregate [codegen id : 2] +(10) CometHashAggregate Input [5]: [count#21, sum#22, count#23, sum#24, count#25] Keys: [] Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#18)), avg(UnscaledValue(ss_net_paid#19))] -Aggregate Attributes [3]: [count(1)#26, avg(UnscaledValue(ss_ext_discount_amt#18))#27, avg(UnscaledValue(ss_net_paid#19))#28] -Results [3]: [count(1)#26 AS count(1)#29, cast((avg(UnscaledValue(ss_ext_discount_amt#18))#27 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#30, cast((avg(UnscaledValue(ss_net_paid#19))#28 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#31] -(12) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#29, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#30, avg(ss_net_paid), avg(ss_net_paid)#31) AS mergedValue#32] -Input [3]: [count(1)#29, avg(ss_ext_discount_amt)#30, avg(ss_net_paid)#31] +(11) CometProject +Input [3]: [count(1)#26, avg(ss_ext_discount_amt)#27, avg(ss_net_paid)#28] +Arguments: [mergedValue#29], [named_struct(count(1), count(1)#26, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#27, avg(ss_net_paid), avg(ss_net_paid)#28) AS mergedValue#29] + +(12) ColumnarToRow [codegen id : 1] +Input [1]: [mergedValue#29] Subquery:2 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] Subquery:3 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#5, [id=#6] -* Project (20) -+- * HashAggregate (19) - +- Exchange (18) - +- * ColumnarToRow (17) +* ColumnarToRow (20) ++- CometProject (19) + +- CometHashAggregate (18) + +- CometColumnarExchange (17) +- CometHashAggregate (16) +- CometProject (15) +- CometFilter (14) @@ -90,52 +88,50 @@ Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (13) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#33, ss_ext_discount_amt#34, ss_net_paid#35, ss_sold_date_sk#36] +Output [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct (14) CometFilter -Input [4]: [ss_quantity#33, ss_ext_discount_amt#34, ss_net_paid#35, ss_sold_date_sk#36] -Condition : ((isnotnull(ss_quantity#33) AND (ss_quantity#33 >= 21)) AND (ss_quantity#33 <= 40)) +Input [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] +Condition : ((isnotnull(ss_quantity#30) AND (ss_quantity#30 >= 21)) AND (ss_quantity#30 <= 40)) (15) CometProject -Input [4]: [ss_quantity#33, ss_ext_discount_amt#34, ss_net_paid#35, ss_sold_date_sk#36] -Arguments: [ss_ext_discount_amt#34, ss_net_paid#35], [ss_ext_discount_amt#34, ss_net_paid#35] +Input [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] +Arguments: [ss_ext_discount_amt#31, ss_net_paid#32], [ss_ext_discount_amt#31, ss_net_paid#32] (16) CometHashAggregate -Input [2]: [ss_ext_discount_amt#34, ss_net_paid#35] +Input [2]: [ss_ext_discount_amt#31, ss_net_paid#32] Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#34)), partial_avg(UnscaledValue(ss_net_paid#35))] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#31)), partial_avg(UnscaledValue(ss_net_paid#32))] -(17) ColumnarToRow [codegen id : 1] -Input [5]: [count#37, sum#38, count#39, sum#40, count#41] +(17) CometColumnarExchange +Input [5]: [count#34, sum#35, count#36, sum#37, count#38] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(18) Exchange -Input [5]: [count#37, sum#38, count#39, sum#40, count#41] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] - -(19) HashAggregate [codegen id : 2] -Input [5]: [count#37, sum#38, count#39, sum#40, count#41] +(18) CometHashAggregate +Input [5]: [count#34, sum#35, count#36, sum#37, count#38] Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#34)), avg(UnscaledValue(ss_net_paid#35))] -Aggregate Attributes [3]: [count(1)#42, avg(UnscaledValue(ss_ext_discount_amt#34))#43, avg(UnscaledValue(ss_net_paid#35))#44] -Results [3]: [count(1)#42 AS count(1)#45, cast((avg(UnscaledValue(ss_ext_discount_amt#34))#43 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#46, cast((avg(UnscaledValue(ss_net_paid#35))#44 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#47] +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#31)), avg(UnscaledValue(ss_net_paid#32))] + +(19) CometProject +Input [3]: [count(1)#39, avg(ss_ext_discount_amt)#40, avg(ss_net_paid)#41] +Arguments: [mergedValue#42], [named_struct(count(1), count(1)#39, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#40, avg(ss_net_paid), avg(ss_net_paid)#41) AS mergedValue#42] -(20) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#45, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#46, avg(ss_net_paid), avg(ss_net_paid)#47) AS mergedValue#48] -Input [3]: [count(1)#45, avg(ss_ext_discount_amt)#46, avg(ss_net_paid)#47] +(20) ColumnarToRow [codegen id : 1] +Input [1]: [mergedValue#42] Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] Subquery:6 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#8, [id=#9] -* Project (28) -+- * HashAggregate (27) - +- Exchange (26) - +- * ColumnarToRow (25) +* ColumnarToRow (28) ++- CometProject (27) + +- CometHashAggregate (26) + +- CometColumnarExchange (25) +- CometHashAggregate (24) +- CometProject (23) +- CometFilter (22) @@ -143,52 +139,50 @@ Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (21) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#49, ss_ext_discount_amt#50, ss_net_paid#51, ss_sold_date_sk#52] +Output [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct (22) CometFilter -Input [4]: [ss_quantity#49, ss_ext_discount_amt#50, ss_net_paid#51, ss_sold_date_sk#52] -Condition : ((isnotnull(ss_quantity#49) AND (ss_quantity#49 >= 41)) AND (ss_quantity#49 <= 60)) +Input [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] +Condition : ((isnotnull(ss_quantity#43) AND (ss_quantity#43 >= 41)) AND (ss_quantity#43 <= 60)) (23) CometProject -Input [4]: [ss_quantity#49, ss_ext_discount_amt#50, ss_net_paid#51, ss_sold_date_sk#52] -Arguments: [ss_ext_discount_amt#50, ss_net_paid#51], [ss_ext_discount_amt#50, ss_net_paid#51] +Input [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] +Arguments: [ss_ext_discount_amt#44, ss_net_paid#45], [ss_ext_discount_amt#44, ss_net_paid#45] (24) CometHashAggregate -Input [2]: [ss_ext_discount_amt#50, ss_net_paid#51] +Input [2]: [ss_ext_discount_amt#44, ss_net_paid#45] Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#50)), partial_avg(UnscaledValue(ss_net_paid#51))] - -(25) ColumnarToRow [codegen id : 1] -Input [5]: [count#53, sum#54, count#55, sum#56, count#57] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#44)), partial_avg(UnscaledValue(ss_net_paid#45))] -(26) Exchange -Input [5]: [count#53, sum#54, count#55, sum#56, count#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] +(25) CometColumnarExchange +Input [5]: [count#47, sum#48, count#49, sum#50, count#51] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(27) HashAggregate [codegen id : 2] -Input [5]: [count#53, sum#54, count#55, sum#56, count#57] +(26) CometHashAggregate +Input [5]: [count#47, sum#48, count#49, sum#50, count#51] Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#50)), avg(UnscaledValue(ss_net_paid#51))] -Aggregate Attributes [3]: [count(1)#58, avg(UnscaledValue(ss_ext_discount_amt#50))#59, avg(UnscaledValue(ss_net_paid#51))#60] -Results [3]: [count(1)#58 AS count(1)#61, cast((avg(UnscaledValue(ss_ext_discount_amt#50))#59 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#62, cast((avg(UnscaledValue(ss_net_paid#51))#60 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#63] +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#44)), avg(UnscaledValue(ss_net_paid#45))] -(28) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#61, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#62, avg(ss_net_paid), avg(ss_net_paid)#63) AS mergedValue#64] -Input [3]: [count(1)#61, avg(ss_ext_discount_amt)#62, avg(ss_net_paid)#63] +(27) CometProject +Input [3]: [count(1)#52, avg(ss_ext_discount_amt)#53, avg(ss_net_paid)#54] +Arguments: [mergedValue#55], [named_struct(count(1), count(1)#52, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#53, avg(ss_net_paid), avg(ss_net_paid)#54) AS mergedValue#55] + +(28) ColumnarToRow [codegen id : 1] +Input [1]: [mergedValue#55] Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] Subquery:9 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* Project (36) -+- * HashAggregate (35) - +- Exchange (34) - +- * ColumnarToRow (33) +* ColumnarToRow (36) ++- CometProject (35) + +- CometHashAggregate (34) + +- CometColumnarExchange (33) +- CometHashAggregate (32) +- CometProject (31) +- CometFilter (30) @@ -196,52 +190,50 @@ Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (29) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#65, ss_ext_discount_amt#66, ss_net_paid#67, ss_sold_date_sk#68] +Output [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct (30) CometFilter -Input [4]: [ss_quantity#65, ss_ext_discount_amt#66, ss_net_paid#67, ss_sold_date_sk#68] -Condition : ((isnotnull(ss_quantity#65) AND (ss_quantity#65 >= 61)) AND (ss_quantity#65 <= 80)) +Input [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] +Condition : ((isnotnull(ss_quantity#56) AND (ss_quantity#56 >= 61)) AND (ss_quantity#56 <= 80)) (31) CometProject -Input [4]: [ss_quantity#65, ss_ext_discount_amt#66, ss_net_paid#67, ss_sold_date_sk#68] -Arguments: [ss_ext_discount_amt#66, ss_net_paid#67], [ss_ext_discount_amt#66, ss_net_paid#67] +Input [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] +Arguments: [ss_ext_discount_amt#57, ss_net_paid#58], [ss_ext_discount_amt#57, ss_net_paid#58] (32) CometHashAggregate -Input [2]: [ss_ext_discount_amt#66, ss_net_paid#67] +Input [2]: [ss_ext_discount_amt#57, ss_net_paid#58] Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#66)), partial_avg(UnscaledValue(ss_net_paid#67))] - -(33) ColumnarToRow [codegen id : 1] -Input [5]: [count#69, sum#70, count#71, sum#72, count#73] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#57)), partial_avg(UnscaledValue(ss_net_paid#58))] -(34) Exchange -Input [5]: [count#69, sum#70, count#71, sum#72, count#73] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +(33) CometColumnarExchange +Input [5]: [count#60, sum#61, count#62, sum#63, count#64] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(35) HashAggregate [codegen id : 2] -Input [5]: [count#69, sum#70, count#71, sum#72, count#73] +(34) CometHashAggregate +Input [5]: [count#60, sum#61, count#62, sum#63, count#64] Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#66)), avg(UnscaledValue(ss_net_paid#67))] -Aggregate Attributes [3]: [count(1)#74, avg(UnscaledValue(ss_ext_discount_amt#66))#75, avg(UnscaledValue(ss_net_paid#67))#76] -Results [3]: [count(1)#74 AS count(1)#77, cast((avg(UnscaledValue(ss_ext_discount_amt#66))#75 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#78, cast((avg(UnscaledValue(ss_net_paid#67))#76 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#79] +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#57)), avg(UnscaledValue(ss_net_paid#58))] + +(35) CometProject +Input [3]: [count(1)#65, avg(ss_ext_discount_amt)#66, avg(ss_net_paid)#67] +Arguments: [mergedValue#68], [named_struct(count(1), count(1)#65, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#66, avg(ss_net_paid), avg(ss_net_paid)#67) AS mergedValue#68] -(36) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#77, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#78, avg(ss_net_paid), avg(ss_net_paid)#79) AS mergedValue#80] -Input [3]: [count(1)#77, avg(ss_ext_discount_amt)#78, avg(ss_net_paid)#79] +(36) ColumnarToRow [codegen id : 1] +Input [1]: [mergedValue#68] Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] Subquery:12 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#14, [id=#15] -* Project (44) -+- * HashAggregate (43) - +- Exchange (42) - +- * ColumnarToRow (41) +* ColumnarToRow (44) ++- CometProject (43) + +- CometHashAggregate (42) + +- CometColumnarExchange (41) +- CometHashAggregate (40) +- CometProject (39) +- CometFilter (38) @@ -249,42 +241,40 @@ Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (37) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#81, ss_ext_discount_amt#82, ss_net_paid#83, ss_sold_date_sk#84] +Output [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct (38) CometFilter -Input [4]: [ss_quantity#81, ss_ext_discount_amt#82, ss_net_paid#83, ss_sold_date_sk#84] -Condition : ((isnotnull(ss_quantity#81) AND (ss_quantity#81 >= 81)) AND (ss_quantity#81 <= 100)) +Input [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] +Condition : ((isnotnull(ss_quantity#69) AND (ss_quantity#69 >= 81)) AND (ss_quantity#69 <= 100)) (39) CometProject -Input [4]: [ss_quantity#81, ss_ext_discount_amt#82, ss_net_paid#83, ss_sold_date_sk#84] -Arguments: [ss_ext_discount_amt#82, ss_net_paid#83], [ss_ext_discount_amt#82, ss_net_paid#83] +Input [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] +Arguments: [ss_ext_discount_amt#70, ss_net_paid#71], [ss_ext_discount_amt#70, ss_net_paid#71] (40) CometHashAggregate -Input [2]: [ss_ext_discount_amt#82, ss_net_paid#83] +Input [2]: [ss_ext_discount_amt#70, ss_net_paid#71] Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#82)), partial_avg(UnscaledValue(ss_net_paid#83))] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#70)), partial_avg(UnscaledValue(ss_net_paid#71))] -(41) ColumnarToRow [codegen id : 1] -Input [5]: [count#85, sum#86, count#87, sum#88, count#89] +(41) CometColumnarExchange +Input [5]: [count#73, sum#74, count#75, sum#76, count#77] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(42) Exchange -Input [5]: [count#85, sum#86, count#87, sum#88, count#89] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5] - -(43) HashAggregate [codegen id : 2] -Input [5]: [count#85, sum#86, count#87, sum#88, count#89] +(42) CometHashAggregate +Input [5]: [count#73, sum#74, count#75, sum#76, count#77] Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#82)), avg(UnscaledValue(ss_net_paid#83))] -Aggregate Attributes [3]: [count(1)#90, avg(UnscaledValue(ss_ext_discount_amt#82))#91, avg(UnscaledValue(ss_net_paid#83))#92] -Results [3]: [count(1)#90 AS count(1)#93, cast((avg(UnscaledValue(ss_ext_discount_amt#82))#91 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#94, cast((avg(UnscaledValue(ss_net_paid#83))#92 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#95] +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#70)), avg(UnscaledValue(ss_net_paid#71))] + +(43) CometProject +Input [3]: [count(1)#78, avg(ss_ext_discount_amt)#79, avg(ss_net_paid)#80] +Arguments: [mergedValue#81], [named_struct(count(1), count(1)#78, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#79, avg(ss_net_paid), avg(ss_net_paid)#80) AS mergedValue#81] -(44) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#93, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#94, avg(ss_net_paid), avg(ss_net_paid)#95) AS mergedValue#96] -Input [3]: [count(1)#93, avg(ss_ext_discount_amt)#94, avg(ss_net_paid)#95] +(44) ColumnarToRow [codegen id : 1] +Input [1]: [mergedValue#81] Subquery:14 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt index 5cb600551..e0d03e2ef 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt @@ -1,78 +1,68 @@ WholeStageCodegen (1) Project Subquery #1 - WholeStageCodegen (2) - Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] - HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] - InputAdapter - Exchange #1 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] + CometHashAggregate [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count,count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] + CometColumnarExchange #1 + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] + CometProject [ss_ext_discount_amt,ss_net_paid] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #1 ReusedSubquery [mergedValue] #1 Subquery #2 - WholeStageCodegen (2) - Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] - HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] - InputAdapter - Exchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] + CometHashAggregate [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count,count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] + CometColumnarExchange #2 + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] + CometProject [ss_ext_discount_amt,ss_net_paid] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #2 ReusedSubquery [mergedValue] #2 Subquery #3 - WholeStageCodegen (2) - Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] - HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] - InputAdapter - Exchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] + CometHashAggregate [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count,count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] + CometColumnarExchange #3 + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] + CometProject [ss_ext_discount_amt,ss_net_paid] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #3 ReusedSubquery [mergedValue] #3 Subquery #4 - WholeStageCodegen (2) - Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] - HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] - InputAdapter - Exchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] + CometHashAggregate [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count,count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] + CometColumnarExchange #4 + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] + CometProject [ss_ext_discount_amt,ss_net_paid] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #4 ReusedSubquery [mergedValue] #4 Subquery #5 - WholeStageCodegen (2) - Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] - HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] - InputAdapter - Exchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] + CometHashAggregate [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count,count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] + CometColumnarExchange #5 + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] + CometProject [ss_ext_discount_amt,ss_net_paid] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #5 ReusedSubquery [mergedValue] #5 ColumnarToRow 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 dbea5e75d..ab17ec1d0 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,9 +1,9 @@ == Physical Plan == * Project (47) +- * BroadcastNestedLoopJoin Inner BuildRight (46) - :- * HashAggregate (25) - : +- Exchange (24) - : +- * ColumnarToRow (23) + :- * ColumnarToRow (25) + : +- CometHashAggregate (24) + : +- CometColumnarExchange (23) : +- CometHashAggregate (22) : +- CometProject (21) : +- CometBroadcastHashJoin (20) @@ -27,9 +27,9 @@ : +- CometFilter (17) : +- CometScan parquet spark_catalog.default.web_page (16) +- BroadcastExchange (45) - +- * HashAggregate (44) - +- Exchange (43) - +- * ColumnarToRow (42) + +- * ColumnarToRow (44) + +- CometHashAggregate (43) + +- CometColumnarExchange (42) +- CometHashAggregate (41) +- CometProject (40) +- CometBroadcastHashJoin (39) @@ -151,114 +151,110 @@ Input: [] Keys: [] Functions [1]: [partial_count(1)] -(23) ColumnarToRow [codegen id : 1] +(23) CometColumnarExchange Input [1]: [count#11] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(24) Exchange -Input [1]: [count#11] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] - -(25) HashAggregate [codegen id : 4] +(24) CometHashAggregate Input [1]: [count#11] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#12] -Results [1]: [count(1)#12 AS amc#13] + +(25) ColumnarToRow [codegen id : 2] +Input [1]: [amc#12] (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] +Output [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] 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 (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)) +Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] +Condition : ((isnotnull(ws_ship_hdemo_sk#14) AND isnotnull(ws_sold_time_sk#13)) AND isnotnull(ws_web_page_sk#15)) (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] +Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] +Arguments: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15], [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15] (29) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#18] +Output [1]: [hd_demo_sk#17] (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 +Left output [3]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15] +Right output [1]: [hd_demo_sk#17] +Arguments: [ws_ship_hdemo_sk#14], [hd_demo_sk#17], Inner, BuildRight (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] +Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, hd_demo_sk#17] +Arguments: [ws_sold_time_sk#13, ws_web_page_sk#15], [ws_sold_time_sk#13, ws_web_page_sk#15] (32) Scan parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#19, t_hour#20] +Output [2]: [t_time_sk#18, t_hour#19] 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 (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)) +Input [2]: [t_time_sk#18, t_hour#19] +Condition : (((isnotnull(t_hour#19) AND (t_hour#19 >= 19)) AND (t_hour#19 <= 20)) AND isnotnull(t_time_sk#18)) (34) CometProject -Input [2]: [t_time_sk#19, t_hour#20] -Arguments: [t_time_sk#19], [t_time_sk#19] +Input [2]: [t_time_sk#18, t_hour#19] +Arguments: [t_time_sk#18], [t_time_sk#18] (35) CometBroadcastExchange -Input [1]: [t_time_sk#19] -Arguments: [t_time_sk#19] +Input [1]: [t_time_sk#18] +Arguments: [t_time_sk#18] (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 +Left output [2]: [ws_sold_time_sk#13, ws_web_page_sk#15] +Right output [1]: [t_time_sk#18] +Arguments: [ws_sold_time_sk#13], [t_time_sk#18], Inner, BuildRight (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] +Input [3]: [ws_sold_time_sk#13, ws_web_page_sk#15, t_time_sk#18] +Arguments: [ws_web_page_sk#15], [ws_web_page_sk#15] (38) ReusedExchange [Reuses operator id: 19] -Output [1]: [wp_web_page_sk#21] +Output [1]: [wp_web_page_sk#20] (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 +Left output [1]: [ws_web_page_sk#15] +Right output [1]: [wp_web_page_sk#20] +Arguments: [ws_web_page_sk#15], [wp_web_page_sk#20], Inner, BuildRight (40) CometProject -Input [2]: [ws_web_page_sk#16, wp_web_page_sk#21] +Input [2]: [ws_web_page_sk#15, wp_web_page_sk#20] (41) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -(42) ColumnarToRow [codegen id : 2] -Input [1]: [count#22] +(42) CometColumnarExchange +Input [1]: [count#21] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(43) Exchange -Input [1]: [count#22] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] - -(44) HashAggregate [codegen id : 3] -Input [1]: [count#22] +(43) CometHashAggregate +Input [1]: [count#21] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#23] -Results [1]: [count(1)#23 AS pmc#24] + +(44) ColumnarToRow [codegen id : 1] +Input [1]: [pmc#22] (45) BroadcastExchange -Input [1]: [pmc#24] +Input [1]: [pmc#22] Arguments: IdentityBroadcastMode, [plan_id=3] -(46) BroadcastNestedLoopJoin [codegen id : 4] +(46) BroadcastNestedLoopJoin [codegen id : 2] Join type: Inner Join condition: None -(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] +(47) Project [codegen id : 2] +Output [1]: [(cast(amc#12 as decimal(15,4)) / cast(pmc#22 as decimal(15,4))) AS am_pm_ratio#23] +Input [2]: [amc#12, pmc#22] 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 44159cc73..65860a550 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,56 +1,52 @@ -WholeStageCodegen (4) +WholeStageCodegen (2) Project [amc,pmc] BroadcastNestedLoopJoin - HashAggregate [count] [count(1),amc,count] + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] - CometProject [ws_sold_time_sk,ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] - CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - CometBroadcastExchange [wp_web_page_sk] #4 - CometProject [wp_web_page_sk] - CometFilter [wp_web_page_sk,wp_char_count] - CometScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] + CometHashAggregate [amc,count,count(1)] + CometColumnarExchange #1 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometProject [ws_web_page_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] + CometProject [ws_sold_time_sk,ws_web_page_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] + CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] + CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] + CometBroadcastExchange [hd_demo_sk] #2 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] + CometBroadcastExchange [t_time_sk] #3 + CometProject [t_time_sk] + CometFilter [t_time_sk,t_hour] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] + CometBroadcastExchange [wp_web_page_sk] #4 + CometProject [wp_web_page_sk] + CometFilter [wp_web_page_sk,wp_char_count] + CometScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - HashAggregate [count] [count(1),pmc,count] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange #6 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] - CometProject [ws_sold_time_sk,ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] - CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #7 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - ReusedExchange [wp_web_page_sk] #4 + CometHashAggregate [pmc,count,count(1)] + CometColumnarExchange #6 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometProject [ws_web_page_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] + CometProject [ws_sold_time_sk,ws_web_page_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] + CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] + CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange [t_time_sk] #7 + CometProject [t_time_sk] + CometFilter [t_time_sk,t_hour] + 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 cd2e8cfb0..836e03f4a 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,9 +1,9 @@ == Physical Plan == -* Sort (41) -+- Exchange (40) - +- * HashAggregate (39) - +- Exchange (38) - +- * ColumnarToRow (37) +* ColumnarToRow (41) ++- CometSort (40) + +- CometColumnarExchange (39) + +- CometHashAggregate (38) + +- CometColumnarExchange (37) +- CometHashAggregate (36) +- CometProject (35) +- CometBroadcastHashJoin (34) @@ -215,27 +215,25 @@ Input [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, cd_mari 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))] -(37) ColumnarToRow [codegen id : 1] +(37) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(39) HashAggregate [codegen id : 2] +(38) CometHashAggregate 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))#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] -(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] +(39) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] + +(40) CometSort +Input [4]: [Call_Center#25, Call_Center_Name#26, Manager#27, Returns_Loss#28] +Arguments: [Call_Center#25, Call_Center_Name#26, Manager#27, Returns_Loss#28], [Returns_Loss#28 DESC NULLS LAST] -(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 +(41) ColumnarToRow [codegen id : 1] +Input [4]: [Call_Center#25, Call_Center_Name#26, Manager#27, Returns_Loss#28] ===== Subqueries ===== 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 232c174cd..ff17b986a 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,55 +1,51 @@ -WholeStageCodegen (3) - Sort [Returns_Loss] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [Returns_Loss] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum,cr_net_loss] - CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status,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 [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status] - CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk] - CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,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 [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss] - CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,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,cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] - CometFilter [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometBroadcastExchange [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] #3 - CometFilter [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] #6 - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk] #7 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #8 - 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 [hd_demo_sk] #9 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] + CometSort [Call_Center,Call_Center_Name,Manager,Returns_Loss] + CometColumnarExchange [Returns_Loss] #1 + CometHashAggregate [Call_Center,Call_Center_Name,Manager,Returns_Loss,cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum,sum(UnscaledValue(cr_net_loss))] + CometColumnarExchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 + CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum,cr_net_loss] + CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status,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 [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status] + CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,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 [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,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,cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] + CometFilter [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] + CometBroadcastExchange [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] #3 + CometFilter [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_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_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] #6 + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + CometBroadcastExchange [ca_address_sk] #7 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #8 + 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 [hd_demo_sk] #9 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] + 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 074fe2b99..7d3423272 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,35 +1,34 @@ == Physical Plan == -* 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) +* ColumnarToRow (30) ++- CometHashAggregate (29) + +- CometColumnarExchange (28) + +- CometHashAggregate (27) + +- CometProject (26) + +- CometBroadcastHashJoin (25) + :- CometProject (23) + : +- CometBroadcastHashJoin (22) + : :- 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) + : +- CometBroadcastExchange (21) + : +- CometFilter (20) + : +- CometHashAggregate (19) + : +- CometColumnarExchange (18) + : +- CometHashAggregate (17) + : +- CometProject (16) + : +- CometBroadcastHashJoin (15) + : :- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.web_sales (9) + : +- CometBroadcastExchange (14) + : +- CometProject (13) + : +- CometFilter (12) + : +- CometScan parquet spark_catalog.default.date_dim (11) + +- ReusedExchange (24) (1) Scan parquet spark_catalog.default.web_sales @@ -72,10 +71,7 @@ Arguments: [ws_item_sk#1], [i_item_sk#5], Inner, BuildRight 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] - -(10) Scan parquet spark_catalog.default.web_sales +(9) 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 [] @@ -83,138 +79,130 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(11) CometFilter +(10) CometFilter Input [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] Condition : isnotnull(ws_item_sk#7) -(12) Scan parquet spark_catalog.default.date_dim +(11) 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 -(13) CometFilter +(12) 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)) -(14) CometProject +(13) CometProject Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(15) CometBroadcastExchange +(14) CometBroadcastExchange Input [1]: [d_date_sk#11] Arguments: [d_date_sk#11] -(16) CometBroadcastHashJoin +(15) 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 +(16) 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) CometHashAggregate +(17) 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))] -(19) ColumnarToRow [codegen id : 1] -Input [3]: [ws_item_sk#7, sum#13, count#14] - -(20) Exchange +(18) CometColumnarExchange Input [3]: [ws_item_sk#7, sum#13, count#14] -Arguments: hashpartitioning(ws_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: hashpartitioning(ws_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(21) HashAggregate [codegen id : 2] +(19) CometHashAggregate 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))#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] -(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) +(20) CometFilter +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] +Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#15) -(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] +(21) CometBroadcastExchange +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] +Arguments: [(1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] -(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))#16) +(22) CometBroadcastHashJoin +Left output [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] +Right output [2]: [(1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] +Arguments: [i_item_sk#5], [ws_item_sk#7], Inner, (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#15), BuildRight -(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))#16, ws_item_sk#7] +(23) CometProject +Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] +Arguments: [ws_ext_discount_amt#2, ws_sold_date_sk#3], [ws_ext_discount_amt#2, ws_sold_date_sk#3] -(26) ReusedExchange [Reuses operator id: 36] -Output [1]: [d_date_sk#17] +(24) ReusedExchange [Reuses operator id: 14] +Output [1]: [d_date_sk#16] -(27) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#17] -Join type: Inner -Join condition: None +(25) CometBroadcastHashJoin +Left output [2]: [ws_ext_discount_amt#2, ws_sold_date_sk#3] +Right output [1]: [d_date_sk#16] +Arguments: [ws_sold_date_sk#3], [d_date_sk#16], Inner, BuildRight -(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#17] +(26) CometProject +Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#16] +Arguments: [ws_ext_discount_amt#2], [ws_ext_discount_amt#2] -(29) HashAggregate [codegen id : 4] +(27) CometHashAggregate Input [1]: [ws_ext_discount_amt#2] Keys: [] Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum#18] -Results [1]: [sum#19] -(30) Exchange -Input [1]: [sum#19] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] +(28) CometColumnarExchange +Input [1]: [sum#17] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(31) HashAggregate [codegen id : 5] -Input [1]: [sum#19] +(29) CometHashAggregate +Input [1]: [sum#17] Keys: [] Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] -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] + +(30) ColumnarToRow [codegen id : 1] +Input [1]: [Excess Discount Amount #18] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (36) -+- * ColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan parquet spark_catalog.default.date_dim (32) +BroadcastExchange (35) ++- * ColumnarToRow (34) + +- CometProject (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.date_dim (31) -(32) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#17, d_date#22] +(31) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#16, d_date#19] 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 -(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) CometFilter +Input [2]: [d_date_sk#16, d_date#19] +Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-01-27)) AND (d_date#19 <= 2000-04-26)) AND isnotnull(d_date_sk#16)) -(34) CometProject -Input [2]: [d_date_sk#17, d_date#22] -Arguments: [d_date_sk#17], [d_date_sk#17] +(33) CometProject +Input [2]: [d_date_sk#16, d_date#19] +Arguments: [d_date_sk#16], [d_date_sk#16] -(35) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#17] +(34) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#16] -(36) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(35) BroadcastExchange +Input [1]: [d_date_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -Subquery:2 Hosting operator id = 10 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 9 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 0681a64bd..065e38bb1 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,50 +1,41 @@ -WholeStageCodegen (5) - HashAggregate [sum] [sum(UnscaledValue(ws_ext_discount_amt)),Excess Discount Amount ,sum] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange #1 - 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))] - ColumnarToRow - InputAdapter - CometProject [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] - CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] - CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk] #3 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - InputAdapter - BroadcastExchange #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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ws_item_sk,sum,count,ws_ext_discount_amt] - CometProject [ws_item_sk,ws_ext_discount_amt] - CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - ReusedExchange [d_date_sk] #2 + CometHashAggregate [Excess Discount Amount ,sum,sum(UnscaledValue(ws_ext_discount_amt))] + CometColumnarExchange #1 + CometHashAggregate [sum,ws_ext_discount_amt] + CometProject [ws_ext_discount_amt] + CometBroadcastHashJoin [ws_ext_discount_amt,ws_sold_date_sk,d_date_sk] + CometProject [ws_ext_discount_amt,ws_sold_date_sk] + CometBroadcastHashJoin [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk,(1.3 * avg(ws_ext_discount_amt)),ws_item_sk] + CometProject [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] + CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk] #3 + CometProject [i_item_sk] + CometFilter [i_item_sk,i_manufact_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + CometBroadcastExchange [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk] #4 + CometFilter [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk] + CometHashAggregate [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk,sum,count,avg(UnscaledValue(ws_ext_discount_amt))] + CometColumnarExchange [ws_item_sk] #5 + CometHashAggregate [ws_item_sk,sum,count,ws_ext_discount_amt] + CometProject [ws_item_sk,ws_ext_discount_amt] + CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt index 00ed822f2..ee3fb07aa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt @@ -1,28 +1,26 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * HashAggregate (23) - +- Exchange (22) - +- * HashAggregate (21) - +- * Project (20) - +- * BroadcastHashJoin Inner BuildRight (19) - :- * Project (13) - : +- * SortMergeJoin Inner (12) - : :- * Sort (5) - : : +- Exchange (4) - : : +- * ColumnarToRow (3) - : : +- CometProject (2) - : : +- CometScan parquet spark_catalog.default.store_sales (1) - : +- * Sort (11) - : +- Exchange (10) - : +- * ColumnarToRow (9) - : +- CometProject (8) - : +- CometFilter (7) - : +- CometScan parquet spark_catalog.default.store_returns (6) - +- BroadcastExchange (18) - +- * ColumnarToRow (17) - +- CometProject (16) - +- CometFilter (15) - +- CometScan parquet spark_catalog.default.reason (14) +* ColumnarToRow (22) ++- CometTakeOrderedAndProject (21) + +- CometHashAggregate (20) + +- CometColumnarExchange (19) + +- CometHashAggregate (18) + +- CometProject (17) + +- CometBroadcastHashJoin (16) + :- CometProject (11) + : +- CometSortMergeJoin (10) + : :- CometSort (4) + : : +- CometColumnarExchange (3) + : : +- CometProject (2) + : : +- CometScan parquet spark_catalog.default.store_sales (1) + : +- CometSort (9) + : +- CometColumnarExchange (8) + : +- CometProject (7) + : +- CometFilter (6) + : +- CometScan parquet spark_catalog.default.store_returns (5) + +- CometBroadcastExchange (15) + +- CometProject (14) + +- CometFilter (13) + +- CometScan parquet spark_catalog.default.reason (12) (1) Scan parquet spark_catalog.default.store_sales @@ -35,104 +33,92 @@ ReadSchema: struct -(7) CometFilter +(6) CometFilter Input [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] Condition : ((isnotnull(sr_item_sk#7) AND isnotnull(sr_ticket_number#9)) AND isnotnull(sr_reason_sk#8)) -(8) CometProject +(7) CometProject Input [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] Arguments: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10], [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -(9) ColumnarToRow [codegen id : 3] -Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] - -(10) Exchange +(8) CometColumnarExchange Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: hashpartitioning(sr_item_sk#7, sr_ticket_number#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(sr_item_sk#7, sr_ticket_number#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(11) Sort [codegen id : 4] +(9) CometSort Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [sr_item_sk#7 ASC NULLS FIRST, sr_ticket_number#9 ASC NULLS FIRST], false, 0 +Arguments: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10], [sr_item_sk#7 ASC NULLS FIRST, sr_ticket_number#9 ASC NULLS FIRST] -(12) SortMergeJoin [codegen id : 6] -Left keys [2]: [ss_item_sk#1, ss_ticket_number#3] -Right keys [2]: [sr_item_sk#7, sr_ticket_number#9] -Join type: Inner -Join condition: None +(10) CometSortMergeJoin +Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] +Right output [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] +Arguments: [ss_item_sk#1, ss_ticket_number#3], [sr_item_sk#7, sr_ticket_number#9], Inner -(13) Project [codegen id : 6] -Output [5]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10] +(11) CometProject Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] +Arguments: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10], [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10] -(14) Scan parquet spark_catalog.default.reason +(12) Scan parquet spark_catalog.default.reason Output [2]: [r_reason_sk#12, r_reason_desc#13] Batched: true Location [not included in comparison]/{warehouse_dir}/reason] PushedFilters: [IsNotNull(r_reason_desc), EqualTo(r_reason_desc,reason 28 ), IsNotNull(r_reason_sk)] ReadSchema: struct -(15) CometFilter +(13) CometFilter Input [2]: [r_reason_sk#12, r_reason_desc#13] Condition : ((isnotnull(r_reason_desc#13) AND (r_reason_desc#13 = reason 28 )) AND isnotnull(r_reason_sk#12)) -(16) CometProject +(14) CometProject Input [2]: [r_reason_sk#12, r_reason_desc#13] Arguments: [r_reason_sk#12], [r_reason_sk#12] -(17) ColumnarToRow [codegen id : 5] +(15) CometBroadcastExchange Input [1]: [r_reason_sk#12] +Arguments: [r_reason_sk#12] -(18) BroadcastExchange -Input [1]: [r_reason_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(19) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_reason_sk#8] -Right keys [1]: [r_reason_sk#12] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [5]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10] +Right output [1]: [r_reason_sk#12] +Arguments: [sr_reason_sk#8], [r_reason_sk#12], Inner, BuildRight -(20) Project [codegen id : 6] -Output [2]: [ss_customer_sk#2, CASE WHEN isnotnull(sr_return_quantity#10) THEN (cast((ss_quantity#4 - sr_return_quantity#10) as decimal(10,0)) * ss_sales_price#5) ELSE (cast(ss_quantity#4 as decimal(10,0)) * ss_sales_price#5) END AS act_sales#14] +(17) CometProject Input [6]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10, r_reason_sk#12] +Arguments: [ss_customer_sk#2, act_sales#14], [ss_customer_sk#2, CASE WHEN isnotnull(sr_return_quantity#10) THEN (cast((ss_quantity#4 - sr_return_quantity#10) as decimal(10,0)) * ss_sales_price#5) ELSE (cast(ss_quantity#4 as decimal(10,0)) * ss_sales_price#5) END AS act_sales#14] -(21) HashAggregate [codegen id : 6] +(18) CometHashAggregate Input [2]: [ss_customer_sk#2, act_sales#14] Keys [1]: [ss_customer_sk#2] Functions [1]: [partial_sum(act_sales#14)] -Aggregate Attributes [2]: [sum#15, isEmpty#16] -Results [3]: [ss_customer_sk#2, sum#17, isEmpty#18] -(22) Exchange -Input [3]: [ss_customer_sk#2, sum#17, isEmpty#18] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(19) CometColumnarExchange +Input [3]: [ss_customer_sk#2, sum#15, isEmpty#16] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) HashAggregate [codegen id : 7] -Input [3]: [ss_customer_sk#2, sum#17, isEmpty#18] +(20) CometHashAggregate +Input [3]: [ss_customer_sk#2, sum#15, isEmpty#16] Keys [1]: [ss_customer_sk#2] Functions [1]: [sum(act_sales#14)] -Aggregate Attributes [1]: [sum(act_sales#14)#19] -Results [2]: [ss_customer_sk#2, sum(act_sales#14)#19 AS sumsales#20] -(24) TakeOrderedAndProject -Input [2]: [ss_customer_sk#2, sumsales#20] -Arguments: 100, [sumsales#20 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], [ss_customer_sk#2, sumsales#20] +(21) CometTakeOrderedAndProject +Input [2]: [ss_customer_sk#2, sumsales#17] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[sumsales#17 ASC NULLS FIRST,ss_customer_sk#2 ASC NULLS FIRST], output=[ss_customer_sk#2,sumsales#17]), [ss_customer_sk#2, sumsales#17], 100, [sumsales#17 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], [ss_customer_sk#2, sumsales#17] + +(22) ColumnarToRow [codegen id : 1] +Input [2]: [ss_customer_sk#2, sumsales#17] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt index a8eb231b7..c4fb926a5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt @@ -1,40 +1,24 @@ -TakeOrderedAndProject [sumsales,ss_customer_sk] - WholeStageCodegen (7) - HashAggregate [ss_customer_sk,sum,isEmpty] [sum(act_sales),sumsales,sum,isEmpty] - InputAdapter - Exchange [ss_customer_sk] #1 - WholeStageCodegen (6) - HashAggregate [ss_customer_sk,act_sales] [sum,isEmpty,sum,isEmpty] - Project [ss_customer_sk,sr_return_quantity,ss_quantity,ss_sales_price] - BroadcastHashJoin [sr_reason_sk,r_reason_sk] - Project [ss_customer_sk,ss_quantity,ss_sales_price,sr_reason_sk,sr_return_quantity] - SortMergeJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - WholeStageCodegen (2) - Sort [ss_item_sk,ss_ticket_number] - InputAdapter - Exchange [ss_item_sk,ss_ticket_number] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] - InputAdapter - WholeStageCodegen (4) - Sort [sr_item_sk,sr_ticket_number] - InputAdapter - Exchange [sr_item_sk,sr_ticket_number] #3 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometFilter [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometProject [r_reason_sk] - CometFilter [r_reason_sk,r_reason_desc] - CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [ss_customer_sk,sumsales] + CometHashAggregate [ss_customer_sk,sumsales,sum,isEmpty,sum(act_sales)] + CometColumnarExchange [ss_customer_sk] #1 + CometHashAggregate [ss_customer_sk,sum,isEmpty,act_sales] + CometProject [sr_return_quantity,ss_quantity,ss_sales_price] [ss_customer_sk,act_sales] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,sr_reason_sk,sr_return_quantity,r_reason_sk] + CometProject [ss_customer_sk,ss_quantity,ss_sales_price,sr_reason_sk,sr_return_quantity] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] + CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] + CometColumnarExchange [ss_item_sk,ss_ticket_number] #2 + CometProject [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometSort [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] + CometColumnarExchange [sr_item_sk,sr_ticket_number] #3 + CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] + CometFilter [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometBroadcastExchange [r_reason_sk] #4 + CometProject [r_reason_sk] + CometFilter [r_reason_sk,r_reason_desc] + 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/q94/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt index d71f96e15..ff1dd24d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt @@ -1,49 +1,51 @@ == Physical Plan == -* HashAggregate (45) -+- Exchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * SortMergeJoin LeftAnti (19) - : : : :- * Project (13) - : : : : +- * SortMergeJoin LeftSemi (12) - : : : : :- * Sort (6) - : : : : : +- Exchange (5) - : : : : : +- * ColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : : +- * Sort (11) - : : : : +- Exchange (10) - : : : : +- * ColumnarToRow (9) - : : : : +- CometProject (8) - : : : : +- CometScan parquet spark_catalog.default.web_sales (7) - : : : +- * Sort (18) - : : : +- Exchange (17) - : : : +- * ColumnarToRow (16) - : : : +- CometProject (15) - : : : +- CometScan parquet spark_catalog.default.web_returns (14) - : : +- BroadcastExchange (24) - : : +- * ColumnarToRow (23) - : : +- CometProject (22) - : : +- CometFilter (21) - : : +- CometScan parquet spark_catalog.default.date_dim (20) - : +- 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.web_site (34) +* HashAggregate (47) ++- * ColumnarToRow (46) + +- CometColumnarExchange (45) + +- RowToColumnar (44) + +- * HashAggregate (43) + +- * HashAggregate (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * SortMergeJoin LeftAnti (19) + : : : :- * Project (13) + : : : : +- * SortMergeJoin LeftSemi (12) + : : : : :- * ColumnarToRow (6) + : : : : : +- CometSort (5) + : : : : : +- CometColumnarExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : : +- * ColumnarToRow (11) + : : : : +- CometSort (10) + : : : : +- CometColumnarExchange (9) + : : : : +- CometProject (8) + : : : : +- CometScan parquet spark_catalog.default.web_sales (7) + : : : +- * ColumnarToRow (18) + : : : +- CometSort (17) + : : : +- CometColumnarExchange (16) + : : : +- CometProject (15) + : : : +- CometScan parquet spark_catalog.default.web_returns (14) + : : +- BroadcastExchange (24) + : : +- * ColumnarToRow (23) + : : +- CometProject (22) + : : +- CometFilter (21) + : : +- CometScan parquet spark_catalog.default.date_dim (20) + : +- 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.web_site (34) (1) Scan parquet spark_catalog.default.web_sales @@ -61,16 +63,16 @@ Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -(4) ColumnarToRow [codegen id : 1] +(4) CometColumnarExchange Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] +Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(5) Exchange +(5) CometSort Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_order_number#5 ASC NULLS FIRST] -(6) Sort [codegen id : 2] +(6) ColumnarToRow [codegen id : 1] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: [ws_order_number#5 ASC NULLS FIRST], false, 0 (7) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] @@ -82,24 +84,24 @@ ReadSchema: struct Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] Arguments: [ws_warehouse_sk#9, ws_order_number#10], [ws_warehouse_sk#9, ws_order_number#10] -(9) ColumnarToRow [codegen id : 3] +(9) CometColumnarExchange Input [2]: [ws_warehouse_sk#9, ws_order_number#10] +Arguments: hashpartitioning(ws_order_number#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(10) Exchange +(10) CometSort Input [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: hashpartitioning(ws_order_number#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: [ws_warehouse_sk#9, ws_order_number#10], [ws_order_number#10 ASC NULLS FIRST] -(11) Sort [codegen id : 4] +(11) ColumnarToRow [codegen id : 2] Input [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: [ws_order_number#10 ASC NULLS FIRST], false, 0 -(12) SortMergeJoin [codegen id : 5] +(12) SortMergeJoin [codegen id : 3] Left keys [1]: [ws_order_number#5] Right keys [1]: [ws_order_number#10] Join type: LeftSemi Join condition: NOT (ws_warehouse_sk#4 = ws_warehouse_sk#9) -(13) Project [codegen id : 5] +(13) Project [codegen id : 3] Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] @@ -113,18 +115,18 @@ ReadSchema: struct Input [2]: [wr_order_number#12, wr_returned_date_sk#13] Arguments: [wr_order_number#12], [wr_order_number#12] -(16) ColumnarToRow [codegen id : 6] +(16) CometColumnarExchange Input [1]: [wr_order_number#12] +Arguments: hashpartitioning(wr_order_number#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(17) Exchange +(17) CometSort Input [1]: [wr_order_number#12] -Arguments: hashpartitioning(wr_order_number#12, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: [wr_order_number#12], [wr_order_number#12 ASC NULLS FIRST] -(18) Sort [codegen id : 7] +(18) ColumnarToRow [codegen id : 4] Input [1]: [wr_order_number#12] -Arguments: [wr_order_number#12 ASC NULLS FIRST], false, 0 -(19) SortMergeJoin [codegen id : 11] +(19) SortMergeJoin [codegen id : 8] Left keys [1]: [ws_order_number#5] Right keys [1]: [wr_order_number#12] Join type: LeftAnti @@ -145,20 +147,20 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-01)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(23) ColumnarToRow [codegen id : 8] +(23) ColumnarToRow [codegen id : 5] Input [1]: [d_date_sk#14] (24) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(25) BroadcastHashJoin [codegen id : 11] +(25) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ws_ship_date_sk#1] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(26) Project [codegen id : 11] +(26) Project [codegen id : 8] Output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#14] @@ -177,20 +179,20 @@ Condition : ((isnotnull(ca_state#17) AND (ca_state#17 = IL)) AND isnotnull(ca_ad Input [2]: [ca_address_sk#16, ca_state#17] Arguments: [ca_address_sk#16], [ca_address_sk#16] -(30) ColumnarToRow [codegen id : 9] +(30) ColumnarToRow [codegen id : 6] Input [1]: [ca_address_sk#16] (31) BroadcastExchange Input [1]: [ca_address_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(32) BroadcastHashJoin [codegen id : 11] +(32) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ws_ship_addr_sk#2] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(33) Project [codegen id : 11] +(33) Project [codegen id : 8] Output [4]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#16] @@ -209,49 +211,55 @@ Condition : ((isnotnull(web_company_name#19) AND (web_company_name#19 = pri Input [2]: [web_site_sk#18, web_company_name#19] Arguments: [web_site_sk#18], [web_site_sk#18] -(37) ColumnarToRow [codegen id : 10] +(37) ColumnarToRow [codegen id : 7] Input [1]: [web_site_sk#18] (38) BroadcastExchange Input [1]: [web_site_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(39) BroadcastHashJoin [codegen id : 11] +(39) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ws_web_site_sk#3] Right keys [1]: [web_site_sk#18] Join type: Inner Join condition: None -(40) Project [codegen id : 11] +(40) Project [codegen id : 8] Output [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [5]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#18] -(41) HashAggregate [codegen id : 11] +(41) HashAggregate [codegen id : 8] Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Keys [1]: [ws_order_number#5] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#20, sum(UnscaledValue(ws_net_profit#7))#21] Results [3]: [ws_order_number#5, sum#22, sum#23] -(42) HashAggregate [codegen id : 11] +(42) HashAggregate [codegen id : 8] Input [3]: [ws_order_number#5, sum#22, sum#23] Keys [1]: [ws_order_number#5] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#20, sum(UnscaledValue(ws_net_profit#7))#21] Results [3]: [ws_order_number#5, sum#22, sum#23] -(43) HashAggregate [codegen id : 11] +(43) HashAggregate [codegen id : 8] Input [3]: [ws_order_number#5, sum#22, sum#23] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#20, sum(UnscaledValue(ws_net_profit#7))#21, count(ws_order_number#5)#24] Results [3]: [sum#22, sum#23, count#25] -(44) Exchange +(44) RowToColumnar Input [3]: [sum#22, sum#23, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(45) HashAggregate [codegen id : 12] +(45) CometColumnarExchange +Input [3]: [sum#22, sum#23, count#25] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(46) ColumnarToRow [codegen id : 9] +Input [3]: [sum#22, sum#23, count#25] + +(47) HashAggregate [codegen id : 9] Input [3]: [sum#22, sum#23, count#25] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt index d54c9e0c9..3f324f9f2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt @@ -1,74 +1,70 @@ -WholeStageCodegen (12) +WholeStageCodegen (9) HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] - InputAdapter - Exchange #1 - WholeStageCodegen (11) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] - Project [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_date_sk,d_date_sk] - SortMergeJoin [ws_order_number,wr_order_number] - InputAdapter - WholeStageCodegen (5) - Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] - InputAdapter - WholeStageCodegen (2) - Sort [ws_order_number] - InputAdapter - Exchange [ws_order_number] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometColumnarExchange #1 + RowToColumnar + WholeStageCodegen (8) + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] + Project [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_date_sk,d_date_sk] + SortMergeJoin [ws_order_number,wr_order_number] + InputAdapter + WholeStageCodegen (3) + Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometSort [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometColumnarExchange [ws_order_number] #2 CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (4) - Sort [ws_order_number] - InputAdapter - Exchange [ws_order_number] #3 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter + InputAdapter + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometSort [ws_warehouse_sk,ws_order_number] + CometColumnarExchange [ws_order_number] #3 CometProject [ws_warehouse_sk,ws_order_number] CometScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (7) - Sort [wr_order_number] - InputAdapter - Exchange [wr_order_number] #4 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter + InputAdapter + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometSort [wr_order_number] + CometColumnarExchange [wr_order_number] #4 CometProject [wr_order_number] CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (8) + BroadcastExchange #6 + WholeStageCodegen (6) ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) + BroadcastExchange #7 + WholeStageCodegen (7) ColumnarToRow InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometProject [web_site_sk] - CometFilter [web_site_sk,web_company_name] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] + CometProject [web_site_sk] + CometFilter [web_site_sk,web_company_name] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt index fd5f78427..e1d16718d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt @@ -1,62 +1,67 @@ == Physical Plan == -* HashAggregate (58) -+- Exchange (57) - +- * HashAggregate (56) - +- * HashAggregate (55) - +- * HashAggregate (54) - +- * Project (53) - +- * BroadcastHashJoin Inner BuildRight (52) - :- * Project (46) - : +- * BroadcastHashJoin Inner BuildRight (45) - : :- * Project (39) - : : +- * BroadcastHashJoin Inner BuildRight (38) - : : :- * SortMergeJoin LeftSemi (32) - : : : :- * SortMergeJoin LeftSemi (17) - : : : : :- * Sort (6) - : : : : : +- Exchange (5) - : : : : : +- * ColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : : +- * Project (16) - : : : : +- * SortMergeJoin Inner (15) - : : : : :- * Sort (12) - : : : : : +- Exchange (11) - : : : : : +- * ColumnarToRow (10) - : : : : : +- CometProject (9) - : : : : : +- CometFilter (8) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (7) - : : : : +- * Sort (14) - : : : : +- ReusedExchange (13) - : : : +- * Project (31) - : : : +- * SortMergeJoin Inner (30) - : : : :- * Sort (23) - : : : : +- Exchange (22) - : : : : +- * ColumnarToRow (21) - : : : : +- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan parquet spark_catalog.default.web_returns (18) - : : : +- * Project (29) - : : : +- * SortMergeJoin Inner (28) - : : : :- * Sort (25) - : : : : +- ReusedExchange (24) - : : : +- * Sort (27) - : : : +- ReusedExchange (26) - : : +- BroadcastExchange (37) - : : +- * ColumnarToRow (36) - : : +- CometProject (35) - : : +- CometFilter (34) - : : +- CometScan parquet spark_catalog.default.date_dim (33) - : +- BroadcastExchange (44) - : +- * ColumnarToRow (43) - : +- CometProject (42) - : +- CometFilter (41) - : +- CometScan parquet spark_catalog.default.customer_address (40) - +- BroadcastExchange (51) - +- * ColumnarToRow (50) - +- CometProject (49) - +- CometFilter (48) - +- CometScan parquet spark_catalog.default.web_site (47) +* HashAggregate (63) ++- * ColumnarToRow (62) + +- CometColumnarExchange (61) + +- RowToColumnar (60) + +- * HashAggregate (59) + +- * HashAggregate (58) + +- * HashAggregate (57) + +- * Project (56) + +- * BroadcastHashJoin Inner BuildRight (55) + :- * Project (49) + : +- * BroadcastHashJoin Inner BuildRight (48) + : :- * Project (42) + : : +- * BroadcastHashJoin Inner BuildRight (41) + : : :- * SortMergeJoin LeftSemi (35) + : : : :- * SortMergeJoin LeftSemi (18) + : : : : :- * ColumnarToRow (6) + : : : : : +- CometSort (5) + : : : : : +- CometColumnarExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : : +- * Project (17) + : : : : +- * SortMergeJoin Inner (16) + : : : : :- * ColumnarToRow (12) + : : : : : +- CometSort (11) + : : : : : +- CometColumnarExchange (10) + : : : : : +- CometProject (9) + : : : : : +- CometFilter (8) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (7) + : : : : +- * ColumnarToRow (15) + : : : : +- CometSort (14) + : : : : +- ReusedExchange (13) + : : : +- * Project (34) + : : : +- * SortMergeJoin Inner (33) + : : : :- * ColumnarToRow (24) + : : : : +- CometSort (23) + : : : : +- CometColumnarExchange (22) + : : : : +- CometProject (21) + : : : : +- CometFilter (20) + : : : : +- CometScan parquet spark_catalog.default.web_returns (19) + : : : +- * Project (32) + : : : +- * SortMergeJoin Inner (31) + : : : :- * ColumnarToRow (27) + : : : : +- CometSort (26) + : : : : +- ReusedExchange (25) + : : : +- * ColumnarToRow (30) + : : : +- CometSort (29) + : : : +- ReusedExchange (28) + : : +- BroadcastExchange (40) + : : +- * ColumnarToRow (39) + : : +- CometProject (38) + : : +- CometFilter (37) + : : +- CometScan parquet spark_catalog.default.date_dim (36) + : +- BroadcastExchange (47) + : +- * ColumnarToRow (46) + : +- CometProject (45) + : +- CometFilter (44) + : +- CometScan parquet spark_catalog.default.customer_address (43) + +- BroadcastExchange (54) + +- * ColumnarToRow (53) + +- CometProject (52) + +- CometFilter (51) + +- CometScan parquet spark_catalog.default.web_site (50) (1) Scan parquet spark_catalog.default.web_sales @@ -74,16 +79,16 @@ Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -(4) ColumnarToRow [codegen id : 1] +(4) CometColumnarExchange Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] +Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(5) Exchange +(5) CometSort Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_order_number#4 ASC NULLS FIRST] -(6) Sort [codegen id : 2] +(6) ColumnarToRow [codegen id : 1] Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Arguments: [ws_order_number#4 ASC NULLS FIRST], false, 0 (7) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] @@ -100,228 +105,243 @@ Condition : (isnotnull(ws_order_number#9) AND isnotnull(ws_warehouse_sk#8)) Input [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_warehouse_sk#8, ws_order_number#9] -(10) ColumnarToRow [codegen id : 3] +(10) CometColumnarExchange Input [2]: [ws_warehouse_sk#8, ws_order_number#9] +Arguments: hashpartitioning(ws_order_number#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(11) Exchange +(11) CometSort Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: hashpartitioning(ws_order_number#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_order_number#9 ASC NULLS FIRST] -(12) Sort [codegen id : 4] +(12) ColumnarToRow [codegen id : 2] Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: [ws_order_number#9 ASC NULLS FIRST], false, 0 -(13) ReusedExchange [Reuses operator id: 11] +(13) ReusedExchange [Reuses operator id: 10] Output [2]: [ws_warehouse_sk#11, ws_order_number#12] -(14) Sort [codegen id : 6] +(14) CometSort Input [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_order_number#12 ASC NULLS FIRST], false, 0 +Arguments: [ws_warehouse_sk#11, ws_order_number#12], [ws_order_number#12 ASC NULLS FIRST] -(15) SortMergeJoin [codegen id : 7] +(15) ColumnarToRow [codegen id : 3] +Input [2]: [ws_warehouse_sk#11, ws_order_number#12] + +(16) SortMergeJoin [codegen id : 4] Left keys [1]: [ws_order_number#9] Right keys [1]: [ws_order_number#12] Join type: Inner Join condition: NOT (ws_warehouse_sk#8 = ws_warehouse_sk#11) -(16) Project [codegen id : 7] +(17) Project [codegen id : 4] Output [1]: [ws_order_number#9] Input [4]: [ws_warehouse_sk#8, ws_order_number#9, ws_warehouse_sk#11, ws_order_number#12] -(17) SortMergeJoin [codegen id : 8] +(18) SortMergeJoin [codegen id : 5] Left keys [1]: [ws_order_number#4] Right keys [1]: [ws_order_number#9] Join type: LeftSemi Join condition: None -(18) Scan parquet spark_catalog.default.web_returns +(19) Scan parquet spark_catalog.default.web_returns Output [2]: [wr_order_number#13, wr_returned_date_sk#14] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number)] ReadSchema: struct -(19) CometFilter +(20) CometFilter Input [2]: [wr_order_number#13, wr_returned_date_sk#14] Condition : isnotnull(wr_order_number#13) -(20) CometProject +(21) CometProject Input [2]: [wr_order_number#13, wr_returned_date_sk#14] Arguments: [wr_order_number#13], [wr_order_number#13] -(21) ColumnarToRow [codegen id : 9] +(22) CometColumnarExchange Input [1]: [wr_order_number#13] +Arguments: hashpartitioning(wr_order_number#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) Exchange +(23) CometSort Input [1]: [wr_order_number#13] -Arguments: hashpartitioning(wr_order_number#13, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: [wr_order_number#13], [wr_order_number#13 ASC NULLS FIRST] -(23) Sort [codegen id : 10] +(24) ColumnarToRow [codegen id : 6] Input [1]: [wr_order_number#13] -Arguments: [wr_order_number#13 ASC NULLS FIRST], false, 0 -(24) ReusedExchange [Reuses operator id: 11] +(25) ReusedExchange [Reuses operator id: 10] Output [2]: [ws_warehouse_sk#8, ws_order_number#9] -(25) Sort [codegen id : 12] +(26) CometSort Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: [ws_order_number#9 ASC NULLS FIRST], false, 0 +Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_order_number#9 ASC NULLS FIRST] -(26) ReusedExchange [Reuses operator id: 11] +(27) ColumnarToRow [codegen id : 7] +Input [2]: [ws_warehouse_sk#8, ws_order_number#9] + +(28) ReusedExchange [Reuses operator id: 10] Output [2]: [ws_warehouse_sk#11, ws_order_number#12] -(27) Sort [codegen id : 14] +(29) CometSort +Input [2]: [ws_warehouse_sk#11, ws_order_number#12] +Arguments: [ws_warehouse_sk#11, ws_order_number#12], [ws_order_number#12 ASC NULLS FIRST] + +(30) ColumnarToRow [codegen id : 8] Input [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_order_number#12 ASC NULLS FIRST], false, 0 -(28) SortMergeJoin [codegen id : 15] +(31) SortMergeJoin [codegen id : 9] Left keys [1]: [ws_order_number#9] Right keys [1]: [ws_order_number#12] Join type: Inner Join condition: NOT (ws_warehouse_sk#8 = ws_warehouse_sk#11) -(29) Project [codegen id : 15] +(32) Project [codegen id : 9] Output [1]: [ws_order_number#9] Input [4]: [ws_warehouse_sk#8, ws_order_number#9, ws_warehouse_sk#11, ws_order_number#12] -(30) SortMergeJoin [codegen id : 16] +(33) SortMergeJoin [codegen id : 10] Left keys [1]: [wr_order_number#13] Right keys [1]: [ws_order_number#9] Join type: Inner Join condition: None -(31) Project [codegen id : 16] +(34) Project [codegen id : 10] Output [1]: [wr_order_number#13] Input [2]: [wr_order_number#13, ws_order_number#9] -(32) SortMergeJoin [codegen id : 20] +(35) SortMergeJoin [codegen id : 14] Left keys [1]: [ws_order_number#4] Right keys [1]: [wr_order_number#13] Join type: LeftSemi Join condition: None -(33) Scan parquet spark_catalog.default.date_dim +(36) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#15, d_date#16] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] ReadSchema: struct -(34) CometFilter +(37) CometFilter Input [2]: [d_date_sk#15, d_date#16] Condition : (((isnotnull(d_date#16) AND (d_date#16 >= 1999-02-01)) AND (d_date#16 <= 1999-04-02)) AND isnotnull(d_date_sk#15)) -(35) CometProject +(38) CometProject Input [2]: [d_date_sk#15, d_date#16] Arguments: [d_date_sk#15], [d_date_sk#15] -(36) ColumnarToRow [codegen id : 17] +(39) ColumnarToRow [codegen id : 11] Input [1]: [d_date_sk#15] -(37) BroadcastExchange +(40) BroadcastExchange Input [1]: [d_date_sk#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(38) BroadcastHashJoin [codegen id : 20] +(41) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ws_ship_date_sk#1] Right keys [1]: [d_date_sk#15] Join type: Inner Join condition: None -(39) Project [codegen id : 20] +(42) Project [codegen id : 14] Output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, d_date_sk#15] -(40) Scan parquet spark_catalog.default.customer_address +(43) Scan parquet spark_catalog.default.customer_address Output [2]: [ca_address_sk#17, ca_state#18] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,IL), IsNotNull(ca_address_sk)] ReadSchema: struct -(41) CometFilter +(44) CometFilter Input [2]: [ca_address_sk#17, ca_state#18] Condition : ((isnotnull(ca_state#18) AND (ca_state#18 = IL)) AND isnotnull(ca_address_sk#17)) -(42) CometProject +(45) CometProject Input [2]: [ca_address_sk#17, ca_state#18] Arguments: [ca_address_sk#17], [ca_address_sk#17] -(43) ColumnarToRow [codegen id : 18] +(46) ColumnarToRow [codegen id : 12] Input [1]: [ca_address_sk#17] -(44) BroadcastExchange +(47) BroadcastExchange Input [1]: [ca_address_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(45) BroadcastHashJoin [codegen id : 20] +(48) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ws_ship_addr_sk#2] Right keys [1]: [ca_address_sk#17] Join type: Inner Join condition: None -(46) Project [codegen id : 20] +(49) Project [codegen id : 14] Output [4]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ca_address_sk#17] -(47) Scan parquet spark_catalog.default.web_site +(50) Scan parquet spark_catalog.default.web_site Output [2]: [web_site_sk#19, web_company_name#20] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] ReadSchema: struct -(48) CometFilter +(51) CometFilter Input [2]: [web_site_sk#19, web_company_name#20] Condition : ((isnotnull(web_company_name#20) AND (web_company_name#20 = pri )) AND isnotnull(web_site_sk#19)) -(49) CometProject +(52) CometProject Input [2]: [web_site_sk#19, web_company_name#20] Arguments: [web_site_sk#19], [web_site_sk#19] -(50) ColumnarToRow [codegen id : 19] +(53) ColumnarToRow [codegen id : 13] Input [1]: [web_site_sk#19] -(51) BroadcastExchange +(54) BroadcastExchange Input [1]: [web_site_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(52) BroadcastHashJoin [codegen id : 20] +(55) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ws_web_site_sk#3] Right keys [1]: [web_site_sk#19] Join type: Inner Join condition: None -(53) Project [codegen id : 20] +(56) Project [codegen id : 14] Output [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Input [5]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, web_site_sk#19] -(54) HashAggregate [codegen id : 20] +(57) HashAggregate [codegen id : 14] Input [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Keys [1]: [ws_order_number#4] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#5)), partial_sum(UnscaledValue(ws_net_profit#6))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#21, sum(UnscaledValue(ws_net_profit#6))#22] Results [3]: [ws_order_number#4, sum#23, sum#24] -(55) HashAggregate [codegen id : 20] +(58) HashAggregate [codegen id : 14] Input [3]: [ws_order_number#4, sum#23, sum#24] Keys [1]: [ws_order_number#4] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#21, sum(UnscaledValue(ws_net_profit#6))#22] Results [3]: [ws_order_number#4, sum#23, sum#24] -(56) HashAggregate [codegen id : 20] +(59) HashAggregate [codegen id : 14] Input [3]: [ws_order_number#4, sum#23, sum#24] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6)), partial_count(distinct ws_order_number#4)] Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#21, sum(UnscaledValue(ws_net_profit#6))#22, count(ws_order_number#4)#25] Results [3]: [sum#23, sum#24, count#26] -(57) Exchange +(60) RowToColumnar +Input [3]: [sum#23, sum#24, count#26] + +(61) CometColumnarExchange +Input [3]: [sum#23, sum#24, count#26] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(62) ColumnarToRow [codegen id : 15] Input [3]: [sum#23, sum#24, count#26] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(58) HashAggregate [codegen id : 21] +(63) HashAggregate [codegen id : 15] Input [3]: [sum#23, sum#24, count#26] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#5)), sum(UnscaledValue(ws_net_profit#6)), count(distinct ws_order_number#4)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt index 178c25f66..2c736e6e5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt @@ -1,102 +1,101 @@ -WholeStageCodegen (21) +WholeStageCodegen (15) HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] - InputAdapter - Exchange #1 - WholeStageCodegen (20) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] - Project [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_date_sk,d_date_sk] - SortMergeJoin [ws_order_number,wr_order_number] - InputAdapter - WholeStageCodegen (8) - SortMergeJoin [ws_order_number,ws_order_number] - InputAdapter - WholeStageCodegen (2) - Sort [ws_order_number] - InputAdapter - Exchange [ws_order_number] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometColumnarExchange #1 + RowToColumnar + WholeStageCodegen (14) + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] + Project [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_date_sk,d_date_sk] + SortMergeJoin [ws_order_number,wr_order_number] + InputAdapter + WholeStageCodegen (5) + SortMergeJoin [ws_order_number,ws_order_number] + InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometSort [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometColumnarExchange [ws_order_number] #2 CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (7) - Project [ws_order_number] - SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] - InputAdapter - WholeStageCodegen (4) - Sort [ws_order_number] - InputAdapter - Exchange [ws_order_number] #3 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter + InputAdapter + WholeStageCodegen (4) + Project [ws_order_number] + SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + InputAdapter + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometSort [ws_warehouse_sk,ws_order_number] + CometColumnarExchange [ws_order_number] #3 CometProject [ws_warehouse_sk,ws_order_number] CometFilter [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (6) - Sort [ws_order_number] - InputAdapter - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - InputAdapter - WholeStageCodegen (16) - Project [wr_order_number] - SortMergeJoin [wr_order_number,ws_order_number] - InputAdapter - WholeStageCodegen (10) - Sort [wr_order_number] - InputAdapter - Exchange [wr_order_number] #4 - WholeStageCodegen (9) - ColumnarToRow - InputAdapter + InputAdapter + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometSort [ws_warehouse_sk,ws_order_number] + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + InputAdapter + WholeStageCodegen (10) + Project [wr_order_number] + SortMergeJoin [wr_order_number,ws_order_number] + InputAdapter + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometSort [wr_order_number] + CometColumnarExchange [wr_order_number] #4 CometProject [wr_order_number] CometFilter [wr_order_number,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] - InputAdapter - WholeStageCodegen (15) - Project [ws_order_number] - SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] - InputAdapter - WholeStageCodegen (12) - Sort [ws_order_number] - InputAdapter - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - InputAdapter - WholeStageCodegen (14) - Sort [ws_order_number] - InputAdapter - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + InputAdapter + WholeStageCodegen (9) + Project [ws_order_number] + SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + InputAdapter + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometSort [ws_warehouse_sk,ws_order_number] + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + InputAdapter + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometSort [ws_warehouse_sk,ws_order_number] + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (17) + BroadcastExchange #6 + WholeStageCodegen (12) ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (18) + BroadcastExchange #7 + WholeStageCodegen (13) ColumnarToRow InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (19) - ColumnarToRow - InputAdapter - CometProject [web_site_sk] - CometFilter [web_site_sk,web_company_name] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] + CometProject [web_site_sk] + CometFilter [web_site_sk,web_company_name] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] 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 4bc24750f..6f9d1a3f2 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,7 +1,7 @@ == Physical Plan == -* HashAggregate (25) -+- Exchange (24) - +- * ColumnarToRow (23) +* ColumnarToRow (25) ++- CometHashAggregate (24) + +- CometColumnarExchange (23) +- CometHashAggregate (22) +- CometProject (21) +- CometBroadcastHashJoin (20) @@ -129,17 +129,15 @@ Input: [] Keys: [] Functions [1]: [partial_count(1)] -(23) ColumnarToRow [codegen id : 1] +(23) CometColumnarExchange Input [1]: [count#12] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(24) Exchange -Input [1]: [count#12] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] - -(25) HashAggregate [codegen id : 2] +(24) CometHashAggregate Input [1]: [count#12] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#13] -Results [1]: [count(1)#13 AS count(1)#14] + +(25) ColumnarToRow [codegen id : 1] +Input [1]: [count(1)#13] 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 e9d33a7f6..b8b3e211d 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,29 +1,27 @@ -WholeStageCodegen (2) - HashAggregate [count] [count(1),count(1),count] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] - CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_store_name] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] + CometHashAggregate [count(1),count,count(1)] + CometColumnarExchange #1 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometBroadcastExchange [hd_demo_sk] #2 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] + CometBroadcastExchange [t_time_sk] #3 + CometProject [t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + CometBroadcastExchange [s_store_sk] #4 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_store_name] + 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 7508405a8..e119edec6 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,30 +1,29 @@ == Physical Plan == -* 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) +* ColumnarToRow (25) ++- CometHashAggregate (24) + +- CometColumnarExchange (23) + +- CometHashAggregate (22) + +- CometProject (21) + +- CometSortMergeJoin (20) + :- CometSort (11) + : +- CometHashAggregate (10) + : +- CometColumnarExchange (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) + +- CometSort (19) + +- CometHashAggregate (18) + +- CometColumnarExchange (17) + +- CometHashAggregate (16) + +- CometProject (15) + +- CometBroadcastHashJoin (14) + :- CometScan parquet spark_catalog.default.catalog_sales (12) + +- ReusedExchange (13) (1) Scan parquet spark_catalog.default.store_sales @@ -67,126 +66,114 @@ Input [2]: [ss_item_sk#1, ss_customer_sk#2] Keys [2]: [ss_customer_sk#2, ss_item_sk#1] Functions: [] -(9) ColumnarToRow [codegen id : 1] +(9) CometColumnarExchange Input [2]: [ss_customer_sk#2, ss_item_sk#1] +Arguments: hashpartitioning(ss_customer_sk#2, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=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] - -(11) HashAggregate [codegen id : 2] +(10) CometHashAggregate 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#7, ss_item_sk#1 AS item_sk#8] -(12) Sort [codegen id : 2] +(11) CometSort Input [2]: [customer_sk#7, item_sk#8] -Arguments: [customer_sk#7 ASC NULLS FIRST, item_sk#8 ASC NULLS FIRST], false, 0 +Arguments: [customer_sk#7, item_sk#8], [customer_sk#7 ASC NULLS FIRST, item_sk#8 ASC NULLS FIRST] -(13) Scan parquet spark_catalog.default.catalog_sales +(12) 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#11), dynamicpruningexpression(cs_sold_date_sk#11 IN dynamicpruning#12)] ReadSchema: struct -(14) ReusedExchange [Reuses operator id: 5] +(13) ReusedExchange [Reuses operator id: 5] Output [1]: [d_date_sk#13] -(15) CometBroadcastHashJoin +(14) 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 -(16) CometProject +(15) 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] -(17) CometHashAggregate +(16) CometHashAggregate Input [2]: [cs_bill_customer_sk#9, cs_item_sk#10] Keys [2]: [cs_bill_customer_sk#9, cs_item_sk#10] Functions: [] -(18) ColumnarToRow [codegen id : 3] +(17) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=2] -(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] - -(20) HashAggregate [codegen id : 4] +(18) 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#9 AS customer_sk#14, cs_item_sk#10 AS item_sk#15] -(21) Sort [codegen id : 4] +(19) CometSort Input [2]: [customer_sk#14, item_sk#15] -Arguments: [customer_sk#14 ASC NULLS FIRST, item_sk#15 ASC NULLS FIRST], false, 0 +Arguments: [customer_sk#14, item_sk#15], [customer_sk#14 ASC NULLS FIRST, item_sk#15 ASC NULLS FIRST] -(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) CometSortMergeJoin +Left output [2]: [customer_sk#7, item_sk#8] +Right output [2]: [customer_sk#14, item_sk#15] +Arguments: [customer_sk#7, item_sk#8], [customer_sk#14, item_sk#15], FullOuter -(23) Project [codegen id : 5] -Output [2]: [customer_sk#7, customer_sk#14] +(21) CometProject Input [4]: [customer_sk#7, item_sk#8, customer_sk#14, item_sk#15] +Arguments: [customer_sk#7, customer_sk#14], [customer_sk#7, customer_sk#14] -(24) HashAggregate [codegen id : 5] +(22) CometHashAggregate Input [2]: [customer_sk#7, customer_sk#14] Keys: [] 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] -(25) Exchange -Input [3]: [sum#19, sum#20, sum#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] +(23) CometColumnarExchange +Input [3]: [sum#16, sum#17, sum#18] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(26) HashAggregate [codegen id : 6] -Input [3]: [sum#19, sum#20, sum#21] +(24) CometHashAggregate +Input [3]: [sum#16, sum#17, sum#18] Keys: [] 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] + +(25) ColumnarToRow [codegen id : 1] +Input [3]: [store_only#19, catalog_only#20, store_and_catalog#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (31) -+- * ColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan parquet spark_catalog.default.date_dim (27) +BroadcastExchange (30) ++- * ColumnarToRow (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan parquet spark_catalog.default.date_dim (26) -(27) Scan parquet spark_catalog.default.date_dim +(26) 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 -(28) CometFilter +(27) 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)) -(29) CometProject +(28) CometProject Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(30) ColumnarToRow [codegen id : 1] +(29) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(31) BroadcastExchange +(30) 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 = 13 Hosting Expression = cs_sold_date_sk#11 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 12 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 41f3e579f..da4461d84 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,48 +1,36 @@ -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] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange #1 - 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 (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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ss_customer_sk,ss_item_sk] - CometProject [ss_item_sk,ss_customer_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,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 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - 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] #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometHashAggregate [cs_bill_customer_sk,cs_item_sk] - CometProject [cs_bill_customer_sk,cs_item_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,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 - ReusedExchange [d_date_sk] #4 + CometHashAggregate [store_only,catalog_only,store_and_catalog,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)] + CometColumnarExchange #1 + CometHashAggregate [sum,sum,sum,customer_sk,customer_sk] + CometProject [customer_sk,customer_sk] + CometSortMergeJoin [customer_sk,item_sk,customer_sk,item_sk] + CometSort [customer_sk,item_sk] + CometHashAggregate [customer_sk,item_sk,ss_customer_sk,ss_item_sk] + CometColumnarExchange [ss_customer_sk,ss_item_sk] #2 + CometHashAggregate [ss_customer_sk,ss_item_sk] + CometProject [ss_item_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,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 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometSort [customer_sk,item_sk] + CometHashAggregate [customer_sk,item_sk,cs_bill_customer_sk,cs_item_sk] + CometColumnarExchange [cs_bill_customer_sk,cs_item_sk] #5 + CometHashAggregate [cs_bill_customer_sk,cs_item_sk] + CometProject [cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,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 + 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 ab7a40acd..0239cc78c 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,28 +1,30 @@ == Physical Plan == -* 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) +* ColumnarToRow (26) ++- CometProject (25) + +- CometSort (24) + +- CometColumnarExchange (23) + +- RowToColumnar (22) + +- * Project (21) + +- Window (20) + +- * ColumnarToRow (19) + +- CometSort (18) + +- CometColumnarExchange (17) + +- CometHashAggregate (16) + +- CometColumnarExchange (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 @@ -94,77 +96,81 @@ Input [6]: [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, 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))] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(16) CometHashAggregate 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] -(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=2] +(17) CometColumnarExchange +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(18) CometSort +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] +Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6], [i_class#9 ASC NULLS FIRST] -(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 +(19) ColumnarToRow [codegen id : 1] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] (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] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] +Arguments: [sum(_w0#15) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#16], [i_class#9] + +(21) Project [codegen id : 2] +Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, ((_w0#15 * 100) / _we0#16) AS revenueratio#17, i_item_id#6] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6, _we0#16] + +(22) RowToColumnar +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17, i_item_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] +(23) CometColumnarExchange +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17, 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#17 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(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=3] +(24) CometSort +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17, i_item_id#6] +Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17, i_item_id#6], [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#17 ASC NULLS FIRST] -(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 +(25) CometProject +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17, i_item_id#6] +Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] -(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] +(26) ColumnarToRow [codegen id : 3] +Input [6]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * ColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan parquet spark_catalog.default.date_dim (25) +BroadcastExchange (31) ++- * ColumnarToRow (30) + +- CometProject (29) + +- CometFilter (28) + +- CometScan parquet spark_catalog.default.date_dim (27) -(25) Scan parquet spark_catalog.default.date_dim +(27) 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 -(26) CometFilter +(28) 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)) -(27) CometProject +(29) CometProject Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(28) ColumnarToRow [codegen id : 1] +(30) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(29) BroadcastExchange +(31) BroadcastExchange Input [1]: [d_date_sk#11] 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 3895cdc78..663bc2e3e 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,42 +1,40 @@ -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 (4) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (3) - Sort [i_class] - InputAdapter - Exchange [i_class] #2 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ss_ext_sales_price] - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_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 #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] +WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] + CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 + RowToColumnar + WholeStageCodegen (2) + Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometColumnarExchange [i_class] #2 + CometHashAggregate [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ss_ext_sales_price] + CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_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 #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + 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 131687112..ceb28b9e8 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,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (28) -+- * HashAggregate (27) - +- Exchange (26) - +- * ColumnarToRow (25) +* ColumnarToRow (28) ++- CometTakeOrderedAndProject (27) + +- CometHashAggregate (26) + +- CometColumnarExchange (25) +- CometHashAggregate (24) +- CometProject (23) +- CometBroadcastHashJoin (22) @@ -145,21 +145,19 @@ Input [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#9, cc_name#11, _groupi 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)] -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(27) HashAggregate [codegen id : 2] +(26) CometHashAggregate 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)#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] -(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] +(27) CometTakeOrderedAndProject +Input [8]: [substr(w_warehouse_name, 1, 20)#20, sm_type#9, cc_name#11, 30 days #21, 31 - 60 days #22, 61 - 90 days #23, 91 - 120 days #24, >120 days #25] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#20 ASC NULLS FIRST,sm_type#9 ASC NULLS FIRST,cc_name#11 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#20,sm_type#9,cc_name#11,30 days #21,31 - 60 days #22,61 - 90 days #23,91 - 120 days #24,>120 days #25]), [substr(w_warehouse_name, 1, 20)#20, sm_type#9, cc_name#11, 30 days #21, 31 - 60 days #22, 61 - 90 days #23, 91 - 120 days #24, >120 days #25], 100, [substr(w_warehouse_name, 1, 20)#20 ASC NULLS FIRST, sm_type#9 ASC NULLS FIRST, cc_name#11 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#20, sm_type#9, cc_name#11, 30 days #21, 31 - 60 days #22, 61 - 90 days #23, 91 - 120 days #24, >120 days #25] + +(28) ColumnarToRow [codegen id : 1] +Input [8]: [substr(w_warehouse_name, 1, 20)#20, sm_type#9, cc_name#11, 30 days #21, 31 - 60 days #22, 61 - 90 days #23, 91 - 120 days #24, >120 days #25] 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 fc63929bb..f5ca9c40a 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,32 +1,30 @@ -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 (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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum,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,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name,d_date_sk] - CometProject [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name] - CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_call_center_sk,cc_name] - CometProject [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type] - CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] - CometProject [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name] - CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name] - CometFilter [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_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] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 - CometFilter [sm_ship_mode_sk,sm_type] - CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange [cc_call_center_sk,cc_name] #4 - CometFilter [cc_call_center_sk,cc_name] - CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] + CometHashAggregate [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,_groupingexpression,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)] + CometColumnarExchange [_groupingexpression,sm_type,cc_name] #1 + CometHashAggregate [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum,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,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name,d_date_sk] + CometProject [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_call_center_sk,cc_name] + CometProject [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] + CometProject [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_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] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 + CometFilter [w_warehouse_sk,w_warehouse_name] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 + CometFilter [sm_ship_mode_sk,sm_type] + CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] + CometBroadcastExchange [cc_call_center_sk,cc_name] #4 + CometFilter [cc_call_center_sk,cc_name] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + 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-spark3_5/q10a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/explain.txt index 43d59eb84..f2de73f90 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (39) -+- * HashAggregate (38) - +- Exchange (37) - +- * ColumnarToRow (36) +* ColumnarToRow (39) ++- CometTakeOrderedAndProject (38) + +- CometHashAggregate (37) + +- CometColumnarExchange (36) +- CometHashAggregate (35) +- CometProject (34) +- CometBroadcastHashJoin (33) @@ -207,23 +207,21 @@ Input [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purch 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)] -(36) ColumnarToRow [codegen id : 1] +(36) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(37) CometHashAggregate 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)#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] -(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] +(38) CometTakeOrderedAndProject +Input [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#32, cd_purchase_estimate#26, cnt2#33, cd_credit_rating#27, cnt3#34, cd_dep_count#28, cnt4#35, cd_dep_employed_count#29, cnt5#36, cd_dep_college_count#30, cnt6#37] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[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], output=[cd_gender#23,cd_marital_status#24,cd_education_status#25,cnt1#32,cd_purchase_estimate#26,cnt2#33,cd_credit_rating#27,cnt3#34,cd_dep_count#28,cnt4#35,cd_dep_employed_count#29,cnt5#36,cd_dep_college_count#30,cnt6#37]), [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#32, cd_purchase_estimate#26, cnt2#33, cd_credit_rating#27, cnt3#34, cd_dep_count#28, cnt4#35, cd_dep_employed_count#29, cnt5#36, cd_dep_college_count#30, cnt6#37], 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#32, cd_purchase_estimate#26, cnt2#33, cd_credit_rating#27, cnt3#34, cd_dep_count#28, cnt4#35, cd_dep_employed_count#29, cnt5#36, cd_dep_college_count#30, cnt6#37] + +(39) ColumnarToRow [codegen id : 1] +Input [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#32, cd_purchase_estimate#26, cnt2#33, cd_credit_rating#27, cnt3#34, cd_dep_count#28, cnt4#35, cd_dep_employed_count#29, cnt5#36, cd_dep_college_count#30, cnt6#37] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/simplified.txt index 22c208f2a..243bdbb91 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/simplified.txt @@ -1,53 +1,51 @@ -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 (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 (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,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,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 [c_current_cdemo_sk] - CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk] - CometProject [c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customer_sk] - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] - CometFilter [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] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [customer_sk] #5 - CometUnion [customer_sk] - CometProject [ws_bill_customer_sk] [customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,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_ship_customer_sk,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 [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - CometBroadcastExchange [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] #7 - CometFilter [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] - 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] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cnt1,cd_purchase_estimate,cnt2,cd_credit_rating,cnt3,cd_dep_count,cnt4,cd_dep_employed_count,cnt5,cd_dep_college_count,cnt6] + CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cnt1,cd_purchase_estimate,cnt2,cd_credit_rating,cnt3,cd_dep_count,cnt4,cd_dep_employed_count,cnt5,cd_dep_college_count,cnt6,count,count(1)] + CometColumnarExchange [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 + 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,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,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 [c_current_cdemo_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk] + CometProject [c_current_cdemo_sk,c_current_addr_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customer_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [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] + CometBroadcastExchange [ss_customer_sk] #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [customer_sk] #5 + CometUnion [customer_sk] + CometProject [ws_bill_customer_sk] [customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,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_ship_customer_sk,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 [ca_address_sk] #6 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_county] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + CometBroadcastExchange [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] #7 + CometFilter [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] + 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-spark3_5/q11/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/explain.txt index f2b239def..0ae630c49 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/explain.txt @@ -1,75 +1,72 @@ == Physical Plan == -TakeOrderedAndProject (71) -+- * Project (70) - +- * BroadcastHashJoin Inner BuildRight (69) - :- * 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) - +- * 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) +* ColumnarToRow (68) ++- CometTakeOrderedAndProject (67) + +- CometProject (66) + +- CometBroadcastHashJoin (65) + :- CometProject (50) + : +- CometBroadcastHashJoin (49) + : :- CometBroadcastHashJoin (33) + : : :- CometFilter (16) + : : : +- CometHashAggregate (15) + : : : +- CometColumnarExchange (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) + : : +- CometBroadcastExchange (32) + : : +- CometHashAggregate (31) + : : +- CometColumnarExchange (30) + : : +- CometHashAggregate (29) + : : +- CometProject (28) + : : +- CometBroadcastHashJoin (27) + : : :- CometProject (23) + : : : +- CometBroadcastHashJoin (22) + : : : :- CometFilter (18) + : : : : +- CometScan parquet spark_catalog.default.customer (17) + : : : +- CometBroadcastExchange (21) + : : : +- CometFilter (20) + : : : +- CometScan parquet spark_catalog.default.store_sales (19) + : : +- CometBroadcastExchange (26) + : : +- CometFilter (25) + : : +- CometScan parquet spark_catalog.default.date_dim (24) + : +- CometBroadcastExchange (48) + : +- CometFilter (47) + : +- CometHashAggregate (46) + : +- CometColumnarExchange (45) + : +- CometHashAggregate (44) + : +- CometProject (43) + : +- CometBroadcastHashJoin (42) + : :- CometProject (40) + : : +- CometBroadcastHashJoin (39) + : : :- CometFilter (35) + : : : +- CometScan parquet spark_catalog.default.customer (34) + : : +- CometBroadcastExchange (38) + : : +- CometFilter (37) + : : +- CometScan parquet spark_catalog.default.web_sales (36) + : +- ReusedExchange (41) + +- CometBroadcastExchange (64) + +- CometHashAggregate (63) + +- CometColumnarExchange (62) + +- CometHashAggregate (61) + +- CometProject (60) + +- CometBroadcastHashJoin (59) + :- CometProject (57) + : +- CometBroadcastHashJoin (56) + : :- CometFilter (52) + : : +- CometScan parquet spark_catalog.default.customer (51) + : +- CometBroadcastExchange (55) + : +- CometFilter (54) + : +- CometScan parquet spark_catalog.default.web_sales (53) + +- ReusedExchange (58) (1) Scan parquet spark_catalog.default.customer @@ -137,337 +134,317 @@ Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_fl 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)))] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(16) HashAggregate [codegen id : 8] +(15) CometHashAggregate 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)))#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] -(17) Filter [codegen id : 8] -Input [2]: [customer_id#18, year_total#19] -Condition : (isnotnull(year_total#19) AND (year_total#19 > 0.00)) +(16) CometFilter +Input [2]: [customer_id#17, year_total#18] +Condition : (isnotnull(year_total#18) AND (year_total#18 > 0.00)) -(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] +(17) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(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)) +(18) CometFilter +Input [8]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26] +Condition : (isnotnull(c_customer_sk#19) AND isnotnull(c_customer_id#20)) -(20) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] +(19) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_customer_sk#27, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#31), dynamicpruningexpression(ss_sold_date_sk#31 IN dynamicpruning#32)] +PartitionFilters: [isnotnull(ss_sold_date_sk#30), dynamicpruningexpression(ss_sold_date_sk#30 IN dynamicpruning#31)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(21) CometFilter -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) +(20) CometFilter +Input [4]: [ss_customer_sk#27, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30] +Condition : isnotnull(ss_customer_sk#27) -(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] +(21) CometBroadcastExchange +Input [4]: [ss_customer_sk#27, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30] +Arguments: [ss_customer_sk#27, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30] -(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 +(22) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26] +Right output [4]: [ss_customer_sk#27, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30] +Arguments: [c_customer_sk#19], [ss_customer_sk#27], Inner, BuildRight -(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] +(23) CometProject +Input [12]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, ss_customer_sk#27, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30] +Arguments: [c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30], [c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30] -(25) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_year#34] +(24) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#32, d_year#33] 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 -(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 - -(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] +(25) CometFilter +Input [2]: [d_date_sk#32, d_year#33] +Condition : ((isnotnull(d_year#33) AND (d_year#33 = 2002)) AND isnotnull(d_date_sk#32)) + +(26) CometBroadcastExchange +Input [2]: [d_date_sk#32, d_year#33] +Arguments: [d_date_sk#32, d_year#33] + +(27) CometBroadcastHashJoin +Left output [10]: [c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30] +Right output [2]: [d_date_sk#32, d_year#33] +Arguments: [ss_sold_date_sk#30], [d_date_sk#32], Inner, BuildRight + +(28) CometProject +Input [12]: [c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30, d_date_sk#32, d_year#33] +Arguments: [c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, ss_ext_discount_amt#28, ss_ext_list_price#29, d_year#33], [c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, ss_ext_discount_amt#28, ss_ext_list_price#29, d_year#33] + +(29) CometHashAggregate +Input [10]: [c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, ss_ext_discount_amt#28, ss_ext_list_price#29, d_year#33] +Keys [8]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#33, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26] +Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#29 - ss_ext_discount_amt#28)))] + +(30) CometColumnarExchange +Input [9]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#33, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, sum#34] +Arguments: hashpartitioning(c_customer_id#20, c_first_name#21, c_last_name#22, d_year#33, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(31) CometHashAggregate +Input [9]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#33, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, sum#34] +Keys [8]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#33, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26] +Functions [1]: [sum(UnscaledValue((ss_ext_list_price#29 - ss_ext_discount_amt#28)))] + +(32) CometBroadcastExchange +Input [5]: [customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38, year_total#39] +Arguments: [customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38, year_total#39] + +(33) CometBroadcastHashJoin +Left output [2]: [customer_id#17, year_total#18] +Right output [5]: [customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38, year_total#39] +Arguments: [customer_id#17], [customer_id#35], Inner, BuildRight + +(34) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(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)) +(35) CometFilter +Input [8]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47] +Condition : (isnotnull(c_customer_sk#40) AND isnotnull(c_customer_id#41)) -(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] +(36) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_bill_customer_sk#48, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51] 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#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#52)] 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) +(37) CometFilter +Input [4]: [ws_bill_customer_sk#48, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51] +Condition : isnotnull(ws_bill_customer_sk#48) + +(38) CometBroadcastExchange +Input [4]: [ws_bill_customer_sk#48, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51] +Arguments: [ws_bill_customer_sk#48, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51] -(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] +(39) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47] +Right output [4]: [ws_bill_customer_sk#48, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51] +Arguments: [c_customer_sk#40], [ws_bill_customer_sk#48], Inner, BuildRight -(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 +(40) CometProject +Input [12]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, ws_bill_customer_sk#48, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51] +Arguments: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51], [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51] -(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] +(41) ReusedExchange [Reuses operator id: 10] +Output [2]: [d_date_sk#53, d_year#54] -(43) ReusedExchange [Reuses operator id: 10] -Output [2]: [d_date_sk#54, d_year#55] +(42) CometBroadcastHashJoin +Left output [10]: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51] +Right output [2]: [d_date_sk#53, d_year#54] +Arguments: [ws_sold_date_sk#51], [d_date_sk#53], Inner, BuildRight -(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 +(43) CometProject +Input [12]: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51, d_date_sk#53, d_year#54] +Arguments: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, ws_ext_discount_amt#49, ws_ext_list_price#50, d_year#54], [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, ws_ext_discount_amt#49, ws_ext_list_price#50, d_year#54] -(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] +(44) CometHashAggregate +Input [10]: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, ws_ext_discount_amt#49, ws_ext_list_price#50, d_year#54] +Keys [8]: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, d_year#54] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#50 - ws_ext_discount_amt#49)))] + +(45) CometColumnarExchange +Input [9]: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, d_year#54, sum#55] +Arguments: hashpartitioning(c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, d_year#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (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] - -(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 - -(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] - -(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] +Input [9]: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, d_year#54, sum#55] +Keys [8]: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, d_year#54] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#50 - ws_ext_discount_amt#49)))] + +(47) CometFilter +Input [2]: [customer_id#56, year_total#57] +Condition : (isnotnull(year_total#57) AND (year_total#57 > 0.00)) + +(48) CometBroadcastExchange +Input [2]: [customer_id#56, year_total#57] +Arguments: [customer_id#56, year_total#57] + +(49) CometBroadcastHashJoin +Left output [7]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38, year_total#39] +Right output [2]: [customer_id#56, year_total#57] +Arguments: [customer_id#17], [customer_id#56], Inner, BuildRight + +(50) CometProject +Input [9]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38, year_total#39, customer_id#56, year_total#57] +Arguments: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38, year_total#39, year_total#57], [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38, year_total#39, year_total#57] + +(51) 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#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)) +(52) 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)) -(56) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#68, ws_ext_discount_amt#69, ws_ext_list_price#70, ws_sold_date_sk#71] +(53) Scan parquet spark_catalog.default.web_sales +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#71), dynamicpruningexpression(ws_sold_date_sk#71 IN dynamicpruning#72)] +PartitionFilters: [isnotnull(ws_sold_date_sk#69), dynamicpruningexpression(ws_sold_date_sk#69 IN dynamicpruning#70)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(57) CometFilter -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) +(54) CometFilter +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) -(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] +(55) 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] + +(56) 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 + +(57) 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] + +(58) ReusedExchange [Reuses operator id: 26] +Output [2]: [d_date_sk#71, d_year#72] (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 +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 (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) ReusedExchange [Reuses operator id: 27] -Output [2]: [d_date_sk#73, d_year#74] - -(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) 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) 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) 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#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 : 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#76, year_total#77] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] - -(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#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 : 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#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] +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] + +(61) 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)))] + +(62) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=4] + +(63) CometHashAggregate +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)))] + +(64) CometBroadcastExchange +Input [2]: [customer_id#74, year_total#75] +Arguments: [customer_id#74, year_total#75] + +(65) CometBroadcastHashJoin +Left output [8]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38, year_total#39, year_total#57] +Right output [2]: [customer_id#74, year_total#75] +Arguments: [customer_id#17], [customer_id#74], Inner, (CASE WHEN (year_total#57 > 0.00) THEN (year_total#75 / year_total#57) ELSE 0E-20 END > CASE WHEN (year_total#18 > 0.00) THEN (year_total#39 / year_total#18) ELSE 0E-20 END), BuildRight + +(66) CometProject +Input [10]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38, year_total#39, year_total#57, customer_id#74, year_total#75] +Arguments: [customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38], [customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38] + +(67) CometTakeOrderedAndProject +Input [4]: [customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#35 ASC NULLS FIRST,customer_first_name#36 ASC NULLS FIRST,customer_last_name#37 ASC NULLS FIRST,customer_email_address#38 ASC NULLS FIRST], output=[customer_id#35,customer_first_name#36,customer_last_name#37,customer_email_address#38]), [customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38], 100, [customer_id#35 ASC NULLS FIRST, customer_first_name#36 ASC NULLS FIRST, customer_last_name#37 ASC NULLS FIRST, customer_email_address#38 ASC NULLS FIRST], [customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38] + +(68) ColumnarToRow [codegen id : 1] +Input [4]: [customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (75) -+- * ColumnarToRow (74) - +- CometFilter (73) - +- CometScan parquet spark_catalog.default.date_dim (72) +BroadcastExchange (72) ++- * ColumnarToRow (71) + +- CometFilter (70) + +- CometScan parquet spark_catalog.default.date_dim (69) -(72) Scan parquet spark_catalog.default.date_dim +(69) 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 -(73) CometFilter +(70) 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)) -(74) ColumnarToRow [codegen id : 1] +(71) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#14, d_year#15] -(75) BroadcastExchange +(72) BroadcastExchange Input [2]: [d_date_sk#14, d_year#15] -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=5] -Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#31 IN dynamicpruning#32 -BroadcastExchange (79) -+- * ColumnarToRow (78) - +- CometFilter (77) - +- CometScan parquet spark_catalog.default.date_dim (76) +Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#31 +BroadcastExchange (76) ++- * ColumnarToRow (75) + +- CometFilter (74) + +- CometScan parquet spark_catalog.default.date_dim (73) -(76) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_year#34] +(73) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#32, d_year#33] 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#33, d_year#34] -Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2002)) AND isnotnull(d_date_sk#33)) +(74) CometFilter +Input [2]: [d_date_sk#32, d_year#33] +Condition : ((isnotnull(d_year#33) AND (d_year#33 = 2002)) AND isnotnull(d_date_sk#32)) -(78) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#33, d_year#34] +(75) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#32, d_year#33] -(79) BroadcastExchange -Input [2]: [d_date_sk#33, d_year#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] +(76) BroadcastExchange +Input [2]: [d_date_sk#32, d_year#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -Subquery:3 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#13 +Subquery:3 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#13 -Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#71 IN dynamicpruning#32 +Subquery:4 Hosting operator id = 53 Hosting Expression = ws_sold_date_sk#69 IN dynamicpruning#31 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/simplified.txt index 25010c5eb..c4811d292 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/simplified.txt @@ -1,106 +1,86 @@ -TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_email_address] - 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] - BroadcastHashJoin [customer_id,customer_id] - BroadcastHashJoin [customer_id,customer_id] - Filter [year_total] - 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 (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,sum,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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_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_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - 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,sum,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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 - CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_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 #8 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - 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] #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,sum,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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 - CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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 #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] #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,sum,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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 - CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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 +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_email_address] + CometProject [customer_id,customer_first_name,customer_last_name,customer_email_address] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total,customer_id,year_total] + CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,customer_id,year_total] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total] + CometFilter [customer_id,year_total] + CometHashAggregate [customer_id,year_total,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)))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + 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 [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_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_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,customer_email_address,year_total] #5 + CometHashAggregate [customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,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)))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #6 + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + 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 [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_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 #8 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [customer_id,year_total] #10 + CometFilter [customer_id,year_total] + CometHashAggregate [customer_id,year_total,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)))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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 + CometBroadcastExchange [customer_id,year_total] #13 + CometHashAggregate [customer_id,year_total,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)))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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-spark3_5/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/explain.txt index 04b5d41b5..fbe5feb3e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/explain.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject (22) +- * Project (21) +- Window (20) - +- * Sort (19) - +- Exchange (18) - +- * HashAggregate (17) - +- Exchange (16) - +- * ColumnarToRow (15) + +- * ColumnarToRow (19) + +- CometSort (18) + +- CometColumnarExchange (17) + +- CometHashAggregate (16) + +- CometColumnarExchange (15) +- CometHashAggregate (14) +- CometProject (13) +- CometBroadcastHashJoin (12) @@ -92,39 +92,37 @@ Input [6]: [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, 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))] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(16) CometHashAggregate 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] -(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=2] +(17) CometColumnarExchange +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(18) CometSort +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] +Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15], [i_class#9 ASC NULLS FIRST] -(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 +(19) ColumnarToRow [codegen id : 1] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] (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] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] +Arguments: [sum(_w0#15) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#16], [i_class#9] -(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] +(21) Project [codegen id : 2] +Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, ((_w0#15 * 100) / _we0#16) AS revenueratio#17] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, _we0#16] (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] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] +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#17 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/simplified.txt index a8b8a9cce..e2df89eec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/simplified.txt @@ -1,38 +1,34 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (4) + WholeStageCodegen (2) Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] InputAdapter Window [_w0,i_class] - WholeStageCodegen (3) - Sort [i_class] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [i_class] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ws_ext_sales_price] - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometColumnarExchange [i_class] #1 + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ws_ext_sales_price] + CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + 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-spark3_5/q14/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/explain.txt index 3b81f0c5d..9fc14889b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/explain.txt @@ -1,88 +1,89 @@ == Physical Plan == -TakeOrderedAndProject (84) -+- * BroadcastHashJoin Inner BuildRight (83) - :- * Filter (66) - : +- * HashAggregate (65) - : +- Exchange (64) - : +- * HashAggregate (63) - : +- * Project (62) - : +- * BroadcastHashJoin Inner BuildRight (61) - : :- * Project (59) - : : +- * BroadcastHashJoin Inner BuildRight (58) - : : :- * BroadcastHashJoin LeftSemi BuildRight (51) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (50) - : : : +- * Project (49) - : : : +- * BroadcastHashJoin Inner BuildRight (48) - : : : :- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.item (4) - : : : +- BroadcastExchange (47) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) - : : : :- * HashAggregate (35) - : : : : +- Exchange (34) - : : : : +- * 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) - : : : +- * 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) - : : : +- CometFilter (53) - : : : +- CometScan parquet spark_catalog.default.item (52) - : : +- ReusedExchange (55) - : +- ReusedExchange (60) - +- BroadcastExchange (82) - +- * Filter (81) - +- * HashAggregate (80) - +- Exchange (79) - +- * HashAggregate (78) - +- * Project (77) - +- * BroadcastHashJoin Inner BuildRight (76) - :- * Project (74) - : +- * BroadcastHashJoin Inner BuildRight (73) - : :- * BroadcastHashJoin LeftSemi BuildRight (71) - : : :- * ColumnarToRow (69) - : : : +- CometFilter (68) - : : : +- CometScan parquet spark_catalog.default.store_sales (67) - : : +- ReusedExchange (70) - : +- ReusedExchange (72) - +- ReusedExchange (75) +* ColumnarToRow (85) ++- CometTakeOrderedAndProject (84) + +- CometBroadcastHashJoin (83) + :- CometFilter (64) + : +- CometHashAggregate (63) + : +- CometColumnarExchange (62) + : +- CometHashAggregate (61) + : +- CometProject (60) + : +- CometBroadcastHashJoin (59) + : :- CometProject (54) + : : +- CometBroadcastHashJoin (53) + : : :- CometBroadcastHashJoin (47) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (46) + : : : +- CometProject (45) + : : : +- CometBroadcastHashJoin (44) + : : : :- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.item (3) + : : : +- CometBroadcastExchange (43) + : : : +- CometBroadcastHashJoin (42) + : : : :- CometHashAggregate (32) + : : : : +- CometColumnarExchange (31) + : : : : +- CometHashAggregate (30) + : : : : +- CometProject (29) + : : : : +- CometBroadcastHashJoin (28) + : : : : :- CometProject (26) + : : : : : +- CometBroadcastHashJoin (25) + : : : : : :- CometFilter (6) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (5) + : : : : : +- CometBroadcastExchange (24) + : : : : : +- CometBroadcastHashJoin (23) + : : : : : :- CometFilter (8) + : : : : : : +- CometScan parquet spark_catalog.default.item (7) + : : : : : +- CometBroadcastExchange (22) + : : : : : +- CometProject (21) + : : : : : +- CometBroadcastHashJoin (20) + : : : : : :- CometProject (15) + : : : : : : +- CometBroadcastHashJoin (14) + : : : : : : :- CometFilter (10) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (9) + : : : : : : +- CometBroadcastExchange (13) + : : : : : : +- CometFilter (12) + : : : : : : +- CometScan parquet spark_catalog.default.item (11) + : : : : : +- CometBroadcastExchange (19) + : : : : : +- CometProject (18) + : : : : : +- CometFilter (17) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (16) + : : : : +- ReusedExchange (27) + : : : +- CometBroadcastExchange (41) + : : : +- CometProject (40) + : : : +- CometBroadcastHashJoin (39) + : : : :- CometProject (37) + : : : : +- CometBroadcastHashJoin (36) + : : : : :- CometFilter (34) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (33) + : : : : +- ReusedExchange (35) + : : : +- ReusedExchange (38) + : : +- CometBroadcastExchange (52) + : : +- CometBroadcastHashJoin (51) + : : :- CometFilter (49) + : : : +- CometScan parquet spark_catalog.default.item (48) + : : +- ReusedExchange (50) + : +- CometBroadcastExchange (58) + : +- CometProject (57) + : +- CometFilter (56) + : +- CometScan parquet spark_catalog.default.date_dim (55) + +- CometBroadcastExchange (82) + +- CometFilter (81) + +- CometHashAggregate (80) + +- CometColumnarExchange (79) + +- CometHashAggregate (78) + +- CometProject (77) + +- CometBroadcastHashJoin (76) + :- CometProject (71) + : +- CometBroadcastHashJoin (70) + : :- CometBroadcastHashJoin (68) + : : :- CometFilter (66) + : : : +- CometScan parquet spark_catalog.default.store_sales (65) + : : +- ReusedExchange (67) + : +- ReusedExchange (69) + +- CometBroadcastExchange (75) + +- CometProject (74) + +- CometFilter (73) + +- CometScan parquet spark_catalog.default.date_dim (72) (1) Scan parquet spark_catalog.default.store_sales @@ -97,24 +98,18 @@ 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 : 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 +(3) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) -(6) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(7) Scan parquet spark_catalog.default.store_sales +(5) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] @@ -122,22 +117,22 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(8) CometFilter +(6) CometFilter Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(9) Scan parquet spark_catalog.default.item +(7) 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 -(10) CometFilter +(8) 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)) -(11) Scan parquet spark_catalog.default.catalog_sales +(9) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] @@ -145,116 +140,111 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(12) CometFilter +(10) CometFilter Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) -(13) Scan parquet spark_catalog.default.item +(11) 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 -(14) CometFilter +(12) 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) -(15) CometBroadcastExchange +(13) 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] -(16) CometBroadcastHashJoin +(14) 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 -(17) CometProject +(15) 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 +(16) 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 -(19) CometFilter +(17) 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 +(18) CometProject Input [2]: [d_date_sk#24, d_year#25] Arguments: [d_date_sk#24], [d_date_sk#24] -(21) CometBroadcastExchange +(19) CometBroadcastExchange Input [1]: [d_date_sk#24] Arguments: [d_date_sk#24] -(22) CometBroadcastHashJoin +(20) 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 +(21) 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] -(24) CometBroadcastExchange +(22) CometBroadcastExchange Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] -(25) CometBroadcastHashJoin +(23) 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 -(26) CometBroadcastExchange +(24) CometBroadcastExchange Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(27) CometBroadcastHashJoin +(25) 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 -(28) CometProject +(26) 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] -(29) ReusedExchange [Reuses operator id: 21] +(27) ReusedExchange [Reuses operator id: 19] Output [1]: [d_date_sk#26] -(30) CometBroadcastHashJoin +(28) 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 -(31) CometProject +(29) 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] -(32) CometHashAggregate +(30) CometHashAggregate Input [3]: [brand_id#27, class_id#28, category_id#29] Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] -(33) ColumnarToRow [codegen id : 1] -Input [3]: [brand_id#27, class_id#28, category_id#29] - -(34) Exchange +(31) CometColumnarExchange 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] +Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(35) HashAggregate [codegen id : 3] +(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#27, class_id#28, category_id#29] -(36) Scan parquet spark_catalog.default.web_sales +(33) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] @@ -262,482 +252,492 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(37) CometFilter +(34) CometFilter Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] Condition : isnotnull(ws_item_sk#30) -(38) ReusedExchange [Reuses operator id: 15] +(35) ReusedExchange [Reuses operator id: 13] Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -(39) CometBroadcastHashJoin +(36) 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) CometProject +(37) 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) ReusedExchange [Reuses operator id: 21] +(38) ReusedExchange [Reuses operator id: 19] Output [1]: [d_date_sk#37] -(42) CometBroadcastHashJoin +(39) 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) CometProject +(40) 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) ColumnarToRow [codegen id : 2] -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(45) BroadcastExchange +(41) CometBroadcastExchange 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] +Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] -(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 +(42) CometBroadcastHashJoin +Left output [3]: [brand_id#27, class_id#28, category_id#29] +Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [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)], [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)], LeftSemi, BuildRight -(47) BroadcastExchange +(43) CometBroadcastExchange 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] +Arguments: [brand_id#27, class_id#28, category_id#29] -(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#27, class_id#28, category_id#29] -Join type: Inner -Join condition: None +(44) CometBroadcastHashJoin +Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Right output [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight -(49) Project [codegen id : 4] -Output [1]: [i_item_sk#6 AS ss_item_sk#38] +(45) CometProject 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] +Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] -(50) BroadcastExchange +(46) CometBroadcastExchange Input [1]: [ss_item_sk#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: [ss_item_sk#38] -(51) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#38] -Join type: LeftSemi -Join condition: None +(47) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Right output [1]: [ss_item_sk#38] +Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight -(52) Scan parquet spark_catalog.default.item +(48) Scan parquet spark_catalog.default.item 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 +(49) CometFilter 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 : 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] +(50) ReusedExchange [Reuses operator id: 46] Output [1]: [ss_item_sk#38] -(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 +(51) CometBroadcastHashJoin +Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Right output [1]: [ss_item_sk#38] +Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight -(57) BroadcastExchange +(52) CometBroadcastExchange 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] +Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -(58) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#39] -Join type: Inner -Join condition: None +(53) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight -(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] +(54) CometProject 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] +Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] + +(55) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#43, d_week_seq#44] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] +ReadSchema: struct + +(56) CometFilter +Input [2]: [d_date_sk#43, d_week_seq#44] +Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = ReusedSubquery Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) + +(57) CometProject +Input [2]: [d_date_sk#43, d_week_seq#44] +Arguments: [d_date_sk#43], [d_date_sk#43] -(60) ReusedExchange [Reuses operator id: 106] -Output [1]: [d_date_sk#43] +(58) CometBroadcastExchange +Input [1]: [d_date_sk#43] +Arguments: [d_date_sk#43] -(61) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#43] -Join type: Inner -Join condition: None +(59) CometBroadcastHashJoin +Left 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] +Right output [1]: [d_date_sk#43] +Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight -(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] +(60) CometProject 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] +Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -(63) HashAggregate [codegen id : 11] +(61) CometHashAggregate 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#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 +(62) CometColumnarExchange 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] +Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(65) HashAggregate [codegen id : 24] +(63) CometHashAggregate 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))#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 : 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)))) +(64) CometFilter +Input [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, 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)))) -(67) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60] +(65) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#60), dynamicpruningexpression(ss_sold_date_sk#60 IN dynamicpruning#61)] +PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(68) CometFilter -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 : 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#62] - -(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#63, i_brand_id#64, i_class_id#65, i_category_id#66] - -(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 : 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: 120] -Output [1]: [d_date_sk#67] - -(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 : 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 : 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#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 : 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 : 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#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 : 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#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] +(66) CometFilter +Input [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] +Condition : isnotnull(ss_item_sk#55) + +(67) ReusedExchange [Reuses operator id: 46] +Output [1]: [ss_item_sk#60] + +(68) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] +Right output [1]: [ss_item_sk#60] +Arguments: [ss_item_sk#55], [ss_item_sk#60], LeftSemi, BuildRight + +(69) ReusedExchange [Reuses operator id: 52] +Output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] + +(70) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] +Right output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] +Arguments: [ss_item_sk#55], [i_item_sk#61], Inner, BuildRight + +(71) CometProject +Input [8]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] +Arguments: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64], [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64] + +(72) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#65, d_week_seq#66] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] +ReadSchema: struct + +(73) CometFilter +Input [2]: [d_date_sk#65, d_week_seq#66] +Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = ReusedSubquery Subquery scalar-subquery#67, [id=#68])) AND isnotnull(d_date_sk#65)) + +(74) CometProject +Input [2]: [d_date_sk#65, d_week_seq#66] +Arguments: [d_date_sk#65], [d_date_sk#65] + +(75) CometBroadcastExchange +Input [1]: [d_date_sk#65] +Arguments: [d_date_sk#65] + +(76) CometBroadcastHashJoin +Left output [6]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64] +Right output [1]: [d_date_sk#65] +Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight + +(77) CometProject +Input [7]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64, d_date_sk#65] +Arguments: [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64], [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64] + +(78) CometHashAggregate +Input [5]: [ss_quantity#56, ss_list_price#57, 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#56 as decimal(10,0)) * ss_list_price#57)), partial_count(1)] + +(79) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=3] + +(80) CometHashAggregate +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#56 as decimal(10,0)) * ss_list_price#57)), count(1)] + +(81) CometFilter +Input [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] +Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) + +(82) CometBroadcastExchange +Input [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] +Arguments: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] + +(83) CometBroadcastHashJoin +Left output [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] +Right output [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] +Arguments: [i_brand_id#40, i_class_id#41, i_category_id#42], [i_brand_id#62, i_class_id#63, i_category_id#64], Inner, BuildRight + +(84) CometTakeOrderedAndProject +Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#50,i_brand_id#40,i_class_id#41,i_category_id#42,sales#51,number_sales#52,channel#72,i_brand_id#62,i_class_id#63,i_category_id#64,sales#73,number_sales#74]), [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74], 100, [i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] + +(85) ColumnarToRow [codegen id : 1] +Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] ===== Subqueries ===== -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#79, ss_list_price#80, ss_sold_date_sk#81] +Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#53, [id=#54] +* ColumnarToRow (102) ++- CometHashAggregate (101) + +- CometColumnarExchange (100) + +- CometHashAggregate (99) + +- CometUnion (98) + :- CometProject (89) + : +- CometBroadcastHashJoin (88) + : :- CometScan parquet spark_catalog.default.store_sales (86) + : +- ReusedExchange (87) + :- CometProject (93) + : +- CometBroadcastHashJoin (92) + : :- CometScan parquet spark_catalog.default.catalog_sales (90) + : +- ReusedExchange (91) + +- CometProject (97) + +- CometBroadcastHashJoin (96) + :- CometScan parquet spark_catalog.default.web_sales (94) + +- ReusedExchange (95) + + +(86) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#81), dynamicpruningexpression(ss_sold_date_sk#81 IN dynamicpruning#82)] +PartitionFilters: [isnotnull(ss_sold_date_sk#77), dynamicpruningexpression(ss_sold_date_sk#77 IN dynamicpruning#78)] ReadSchema: struct -(86) ReusedExchange [Reuses operator id: 21] -Output [1]: [d_date_sk#83] +(87) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#79] -(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 +(88) CometBroadcastHashJoin +Left output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] +Right output [1]: [d_date_sk#79] +Arguments: [ss_sold_date_sk#77], [d_date_sk#79], Inner, BuildRight -(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] +(89) CometProject +Input [4]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77, d_date_sk#79] +Arguments: [quantity#80, list_price#81], [ss_quantity#75 AS quantity#80, ss_list_price#76 AS list_price#81] -(89) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#86, cs_list_price#87, cs_sold_date_sk#88] +(90) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#88), dynamicpruningexpression(cs_sold_date_sk#88 IN dynamicpruning#89)] +PartitionFilters: [isnotnull(cs_sold_date_sk#84), dynamicpruningexpression(cs_sold_date_sk#84 IN dynamicpruning#85)] ReadSchema: struct -(90) ReusedExchange [Reuses operator id: 21] -Output [1]: [d_date_sk#90] +(91) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#86] -(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 +(92) CometBroadcastHashJoin +Left output [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] +Right output [1]: [d_date_sk#86] +Arguments: [cs_sold_date_sk#84], [d_date_sk#86], Inner, BuildRight -(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] +(93) CometProject +Input [4]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84, d_date_sk#86] +Arguments: [quantity#87, list_price#88], [cs_quantity#82 AS quantity#87, cs_list_price#83 AS list_price#88] -(93) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#93, ws_list_price#94, ws_sold_date_sk#95] +(94) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#95), dynamicpruningexpression(ws_sold_date_sk#95 IN dynamicpruning#96)] +PartitionFilters: [isnotnull(ws_sold_date_sk#91), dynamicpruningexpression(ws_sold_date_sk#91 IN dynamicpruning#92)] ReadSchema: struct -(94) ReusedExchange [Reuses operator id: 21] -Output [1]: [d_date_sk#97] +(95) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#93] -(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 +(96) CometBroadcastHashJoin +Left output [3]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91] +Right output [1]: [d_date_sk#93] +Arguments: [ws_sold_date_sk#91], [d_date_sk#93], Inner, BuildRight -(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] +(97) CometProject +Input [4]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91, d_date_sk#93] +Arguments: [quantity#94, list_price#95], [ws_quantity#89 AS quantity#94, ws_list_price#90 AS list_price#95] -(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] +(98) CometUnion +Child 0 Input [2]: [quantity#80, list_price#81] +Child 1 Input [2]: [quantity#87, list_price#88] +Child 2 Input [2]: [quantity#94, list_price#95] -(98) CometHashAggregate -Input [2]: [quantity#84, list_price#85] +(99) CometHashAggregate +Input [2]: [quantity#80, list_price#81] Keys: [] -Functions [1]: [partial_avg((cast(quantity#84 as decimal(10,0)) * list_price#85))] +Functions [1]: [partial_avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] -(99) ColumnarToRow [codegen id : 1] -Input [2]: [sum#100, count#101] +(100) CometColumnarExchange +Input [2]: [sum#96, count#97] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(100) Exchange -Input [2]: [sum#100, count#101] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] - -(101) HashAggregate [codegen id : 2] -Input [2]: [sum#100, count#101] +(101) CometHashAggregate +Input [2]: [sum#96, count#97] Keys: [] -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))#102] -Results [1]: [avg((cast(quantity#84 as decimal(10,0)) * list_price#85))#102 AS average_sales#103] +Functions [1]: [avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] + +(102) ColumnarToRow [codegen id : 1] +Input [1]: [average_sales#98] -Subquery:2 Hosting operator id = 85 Hosting Expression = ss_sold_date_sk#81 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 86 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 89 Hosting Expression = cs_sold_date_sk#88 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#84 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 93 Hosting Expression = ws_sold_date_sk#95 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#91 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (106) -+- * ColumnarToRow (105) - +- CometProject (104) - +- CometFilter (103) - +- CometScan parquet spark_catalog.default.date_dim (102) +BroadcastExchange (107) ++- * ColumnarToRow (106) + +- CometProject (105) + +- CometFilter (104) + +- CometScan parquet spark_catalog.default.date_dim (103) -(102) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_week_seq#104] +(103) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#43, d_week_seq#44] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(103) CometFilter -Input [2]: [d_date_sk#43, 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#43)) +(104) CometFilter +Input [2]: [d_date_sk#43, d_week_seq#44] +Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) -(104) CometProject -Input [2]: [d_date_sk#43, d_week_seq#104] +(105) CometProject +Input [2]: [d_date_sk#43, d_week_seq#44] Arguments: [d_date_sk#43], [d_date_sk#43] -(105) ColumnarToRow [codegen id : 1] +(106) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] -(106) BroadcastExchange +(107) BroadcastExchange Input [1]: [d_date_sk#43] -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=5] -Subquery:6 Hosting operator id = 103 Hosting Expression = Subquery scalar-subquery#105, [id=#106] -* ColumnarToRow (110) -+- CometProject (109) - +- CometFilter (108) - +- CometScan parquet spark_catalog.default.date_dim (107) +Subquery:6 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] +* ColumnarToRow (111) ++- CometProject (110) + +- CometFilter (109) + +- CometScan parquet spark_catalog.default.date_dim (108) -(107) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#107, d_year#108, d_moy#109, d_dom#110] +(108) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] 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 -(108) 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)) +(109) CometFilter +Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] +Condition : (((((isnotnull(d_year#100) AND isnotnull(d_moy#101)) AND isnotnull(d_dom#102)) AND (d_year#100 = 1999)) AND (d_moy#101 = 12)) AND (d_dom#102 = 16)) -(109) 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] +(110) CometProject +Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] +Arguments: [d_week_seq#99], [d_week_seq#99] -(110) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#107] +(111) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#99] -Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (115) -+- * ColumnarToRow (114) - +- CometProject (113) - +- CometFilter (112) - +- CometScan parquet spark_catalog.default.date_dim (111) +Subquery:7 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 +BroadcastExchange (116) ++- * ColumnarToRow (115) + +- CometProject (114) + +- CometFilter (113) + +- CometScan parquet spark_catalog.default.date_dim (112) -(111) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#111] +(112) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_year#103] 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 -(112) CometFilter -Input [2]: [d_date_sk#26, d_year#111] -Condition : (((isnotnull(d_year#111) AND (d_year#111 >= 1998)) AND (d_year#111 <= 2000)) AND isnotnull(d_date_sk#26)) +(113) CometFilter +Input [2]: [d_date_sk#26, d_year#103] +Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1998)) AND (d_year#103 <= 2000)) AND isnotnull(d_date_sk#26)) -(113) CometProject -Input [2]: [d_date_sk#26, d_year#111] +(114) CometProject +Input [2]: [d_date_sk#26, d_year#103] Arguments: [d_date_sk#26], [d_date_sk#26] -(114) ColumnarToRow [codegen id : 1] +(115) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] -(115) BroadcastExchange +(116) BroadcastExchange Input [1]: [d_date_sk#26] -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=6] -Subquery:8 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 +Subquery:9 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 -Subquery:10 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#55, [id=#56] +Subquery:10 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -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) +Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#53, [id=#54] +Subquery:12 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 +BroadcastExchange (121) ++- * ColumnarToRow (120) + +- CometProject (119) + +- CometFilter (118) + +- CometScan parquet spark_catalog.default.date_dim (117) -(116) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#67, d_week_seq#112] + +(117) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#65, d_week_seq#66] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(117) CometFilter -Input [2]: [d_date_sk#67, 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#67)) +(118) CometFilter +Input [2]: [d_date_sk#65, d_week_seq#66] +Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = Subquery scalar-subquery#67, [id=#68])) AND isnotnull(d_date_sk#65)) -(118) CometProject -Input [2]: [d_date_sk#67, d_week_seq#112] -Arguments: [d_date_sk#67], [d_date_sk#67] +(119) CometProject +Input [2]: [d_date_sk#65, d_week_seq#66] +Arguments: [d_date_sk#65], [d_date_sk#65] -(119) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#67] +(120) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#65] -(120) BroadcastExchange -Input [1]: [d_date_sk#67] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] +(121) BroadcastExchange +Input [1]: [d_date_sk#65] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:12 Hosting operator id = 117 Hosting Expression = Subquery scalar-subquery#113, [id=#114] -* ColumnarToRow (124) -+- CometProject (123) - +- CometFilter (122) - +- CometScan parquet spark_catalog.default.date_dim (121) +Subquery:13 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#67, [id=#68] +* ColumnarToRow (125) ++- CometProject (124) + +- CometFilter (123) + +- CometScan parquet spark_catalog.default.date_dim (122) -(121) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#115, d_year#116, d_moy#117, d_dom#118] +(122) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] 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 -(122) 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)) +(123) CometFilter +Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] +Condition : (((((isnotnull(d_year#105) AND isnotnull(d_moy#106)) AND isnotnull(d_dom#107)) AND (d_year#105 = 1998)) AND (d_moy#106 = 12)) AND (d_dom#107 = 16)) + +(124) CometProject +Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] +Arguments: [d_week_seq#104], [d_week_seq#104] -(123) 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] +(125) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#104] -(124) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#115] +Subquery:14 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/simplified.txt index aa49638c7..a51d1d007 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/simplified.txt @@ -1,15 +1,15 @@ -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 (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 (2) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - InputAdapter - Exchange #13 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + CometBroadcastHashJoin [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Subquery #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] + CometColumnarExchange #14 CometHashAggregate [sum,count,quantity,list_price] CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] @@ -27,152 +27,127 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ 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 (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] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (4) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #1 + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ss_quantity,ss_list_price] + CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) 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 #4 - 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 (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,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 - CometBroadcastHashJoin [i_item_sk,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 [i_brand_id,i_class_id,i_category_id] #8 - CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [cs_item_sk,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - 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 [d_date_sk] #10 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #10 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (2) + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometBroadcastExchange [ss_item_sk] #3 + CometProject [i_item_sk] [ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,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 [brand_id,class_id,category_id] #4 + CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + CometHashAggregate [brand_id,class_id,category_id] + CometColumnarExchange [brand_id,class_id,category_id] #5 + 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,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 + CometBroadcastHashJoin [i_item_sk,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 [i_brand_id,i_class_id,i_category_id] #8 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + 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 [d_date_sk] #10 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #10 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #11 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + ReusedExchange [d_date_sk] #10 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] + 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] + ReusedExchange [ss_item_sk] #3 + CometBroadcastExchange [d_date_sk] #13 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #2 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometBroadcastExchange [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] #15 + CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + ReusedSubquery [average_sales] #4 + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #16 + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ss_quantity,ss_list_price] + CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_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 #17 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #6 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ws_item_sk,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - ReusedExchange [d_date_sk] #10 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - BroadcastHashJoin [i_item_sk,ss_item_sk] - 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 - ReusedExchange [ss_item_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - 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] #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] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_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 #16 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - ReusedExchange [ss_item_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - InputAdapter - ReusedExchange [d_date_sk] #16 + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + ReusedExchange [ss_item_sk] #3 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + CometBroadcastExchange [d_date_sk] #18 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #6 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/explain.txt index e5dafffc1..832fa1381 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/explain.txt @@ -1,129 +1,126 @@ == Physical Plan == -TakeOrderedAndProject (125) -+- * HashAggregate (124) - +- Exchange (123) - +- * HashAggregate (122) - +- Union (121) - :- * HashAggregate (100) - : +- Exchange (99) - : +- * HashAggregate (98) - : +- Union (97) - : :- * Filter (66) - : : +- * HashAggregate (65) - : : +- Exchange (64) - : : +- * HashAggregate (63) - : : +- * Project (62) - : : +- * BroadcastHashJoin Inner BuildRight (61) - : : :- * Project (59) - : : : +- * BroadcastHashJoin Inner BuildRight (58) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (51) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- BroadcastExchange (50) - : : : : +- * Project (49) - : : : : +- * BroadcastHashJoin Inner BuildRight (48) - : : : : :- * ColumnarToRow (6) - : : : : : +- CometFilter (5) - : : : : : +- CometScan parquet spark_catalog.default.item (4) - : : : : +- BroadcastExchange (47) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) - : : : : :- * HashAggregate (35) - : : : : : +- Exchange (34) - : : : : : +- * 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) - : : : : +- * 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) - : : : : +- CometFilter (53) - : : : : +- CometScan parquet spark_catalog.default.item (52) - : : : +- ReusedExchange (55) - : : +- ReusedExchange (60) - : :- * Filter (81) - : : +- * HashAggregate (80) - : : +- Exchange (79) - : : +- * HashAggregate (78) - : : +- * Project (77) - : : +- * BroadcastHashJoin Inner BuildRight (76) - : : :- * Project (74) - : : : +- * BroadcastHashJoin Inner BuildRight (73) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (71) - : : : : :- * ColumnarToRow (69) - : : : : : +- CometFilter (68) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (67) - : : : : +- ReusedExchange (70) - : : : +- ReusedExchange (72) - : : +- ReusedExchange (75) - : +- * Filter (96) - : +- * HashAggregate (95) - : +- Exchange (94) - : +- * HashAggregate (93) - : +- * Project (92) - : +- * BroadcastHashJoin Inner BuildRight (91) - : :- * Project (89) - : : +- * BroadcastHashJoin Inner BuildRight (88) - : : :- * BroadcastHashJoin LeftSemi BuildRight (86) - : : : :- * ColumnarToRow (84) - : : : : +- CometFilter (83) - : : : : +- CometScan parquet spark_catalog.default.web_sales (82) - : : : +- ReusedExchange (85) - : : +- ReusedExchange (87) - : +- ReusedExchange (90) - :- * HashAggregate (105) - : +- Exchange (104) - : +- * HashAggregate (103) - : +- * HashAggregate (102) - : +- ReusedExchange (101) - :- * HashAggregate (110) - : +- Exchange (109) - : +- * HashAggregate (108) - : +- * HashAggregate (107) - : +- ReusedExchange (106) - :- * HashAggregate (115) - : +- Exchange (114) - : +- * HashAggregate (113) - : +- * HashAggregate (112) - : +- ReusedExchange (111) - +- * HashAggregate (120) - +- Exchange (119) - +- * HashAggregate (118) - +- * HashAggregate (117) - +- ReusedExchange (116) +* ColumnarToRow (122) ++- CometTakeOrderedAndProject (121) + +- CometHashAggregate (120) + +- CometColumnarExchange (119) + +- CometHashAggregate (118) + +- CometUnion (117) + :- CometHashAggregate (96) + : +- CometColumnarExchange (95) + : +- CometHashAggregate (94) + : +- CometUnion (93) + : :- CometFilter (64) + : : +- CometHashAggregate (63) + : : +- CometColumnarExchange (62) + : : +- CometHashAggregate (61) + : : +- CometProject (60) + : : +- CometBroadcastHashJoin (59) + : : :- CometProject (54) + : : : +- CometBroadcastHashJoin (53) + : : : :- CometBroadcastHashJoin (47) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometBroadcastExchange (46) + : : : : +- CometProject (45) + : : : : +- CometBroadcastHashJoin (44) + : : : : :- CometFilter (4) + : : : : : +- CometScan parquet spark_catalog.default.item (3) + : : : : +- CometBroadcastExchange (43) + : : : : +- CometBroadcastHashJoin (42) + : : : : :- CometHashAggregate (32) + : : : : : +- CometColumnarExchange (31) + : : : : : +- CometHashAggregate (30) + : : : : : +- CometProject (29) + : : : : : +- CometBroadcastHashJoin (28) + : : : : : :- CometProject (26) + : : : : : : +- CometBroadcastHashJoin (25) + : : : : : : :- CometFilter (6) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (5) + : : : : : : +- CometBroadcastExchange (24) + : : : : : : +- CometBroadcastHashJoin (23) + : : : : : : :- CometFilter (8) + : : : : : : : +- CometScan parquet spark_catalog.default.item (7) + : : : : : : +- CometBroadcastExchange (22) + : : : : : : +- CometProject (21) + : : : : : : +- CometBroadcastHashJoin (20) + : : : : : : :- CometProject (15) + : : : : : : : +- CometBroadcastHashJoin (14) + : : : : : : : :- CometFilter (10) + : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (9) + : : : : : : : +- CometBroadcastExchange (13) + : : : : : : : +- CometFilter (12) + : : : : : : : +- CometScan parquet spark_catalog.default.item (11) + : : : : : : +- CometBroadcastExchange (19) + : : : : : : +- CometProject (18) + : : : : : : +- CometFilter (17) + : : : : : : +- CometScan parquet spark_catalog.default.date_dim (16) + : : : : : +- ReusedExchange (27) + : : : : +- CometBroadcastExchange (41) + : : : : +- CometProject (40) + : : : : +- CometBroadcastHashJoin (39) + : : : : :- CometProject (37) + : : : : : +- CometBroadcastHashJoin (36) + : : : : : :- CometFilter (34) + : : : : : : +- CometScan parquet spark_catalog.default.web_sales (33) + : : : : : +- ReusedExchange (35) + : : : : +- ReusedExchange (38) + : : : +- CometBroadcastExchange (52) + : : : +- CometBroadcastHashJoin (51) + : : : :- CometFilter (49) + : : : : +- CometScan parquet spark_catalog.default.item (48) + : : : +- ReusedExchange (50) + : : +- CometBroadcastExchange (58) + : : +- CometProject (57) + : : +- CometFilter (56) + : : +- CometScan parquet spark_catalog.default.date_dim (55) + : :- CometFilter (78) + : : +- CometHashAggregate (77) + : : +- CometColumnarExchange (76) + : : +- CometHashAggregate (75) + : : +- CometProject (74) + : : +- CometBroadcastHashJoin (73) + : : :- CometProject (71) + : : : +- CometBroadcastHashJoin (70) + : : : :- CometBroadcastHashJoin (68) + : : : : :- CometFilter (66) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (65) + : : : : +- ReusedExchange (67) + : : : +- ReusedExchange (69) + : : +- ReusedExchange (72) + : +- CometFilter (92) + : +- CometHashAggregate (91) + : +- CometColumnarExchange (90) + : +- CometHashAggregate (89) + : +- CometProject (88) + : +- CometBroadcastHashJoin (87) + : :- CometProject (85) + : : +- CometBroadcastHashJoin (84) + : : :- CometBroadcastHashJoin (82) + : : : :- CometFilter (80) + : : : : +- CometScan parquet spark_catalog.default.web_sales (79) + : : : +- ReusedExchange (81) + : : +- ReusedExchange (83) + : +- ReusedExchange (86) + :- CometHashAggregate (101) + : +- CometColumnarExchange (100) + : +- CometHashAggregate (99) + : +- CometHashAggregate (98) + : +- ReusedExchange (97) + :- CometHashAggregate (106) + : +- CometColumnarExchange (105) + : +- CometHashAggregate (104) + : +- CometHashAggregate (103) + : +- ReusedExchange (102) + :- CometHashAggregate (111) + : +- CometColumnarExchange (110) + : +- CometHashAggregate (109) + : +- CometHashAggregate (108) + : +- ReusedExchange (107) + +- CometHashAggregate (116) + +- CometColumnarExchange (115) + +- CometHashAggregate (114) + +- CometHashAggregate (113) + +- ReusedExchange (112) (1) Scan parquet spark_catalog.default.store_sales @@ -138,24 +135,18 @@ 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 : 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 +(3) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) -(6) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(7) Scan parquet spark_catalog.default.store_sales +(5) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] @@ -163,22 +154,22 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(8) CometFilter +(6) CometFilter Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(9) Scan parquet spark_catalog.default.item +(7) 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 -(10) CometFilter +(8) 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)) -(11) Scan parquet spark_catalog.default.catalog_sales +(9) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] @@ -186,116 +177,111 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(12) CometFilter +(10) CometFilter Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) -(13) Scan parquet spark_catalog.default.item +(11) 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 -(14) CometFilter +(12) 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) -(15) CometBroadcastExchange +(13) 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] -(16) CometBroadcastHashJoin +(14) 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 -(17) CometProject +(15) 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 +(16) 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 -(19) CometFilter +(17) 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 +(18) CometProject Input [2]: [d_date_sk#24, d_year#25] Arguments: [d_date_sk#24], [d_date_sk#24] -(21) CometBroadcastExchange +(19) CometBroadcastExchange Input [1]: [d_date_sk#24] Arguments: [d_date_sk#24] -(22) CometBroadcastHashJoin +(20) 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 +(21) 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] -(24) CometBroadcastExchange +(22) CometBroadcastExchange Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] -(25) CometBroadcastHashJoin +(23) 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 -(26) CometBroadcastExchange +(24) CometBroadcastExchange Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(27) CometBroadcastHashJoin +(25) 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 -(28) CometProject +(26) 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] -(29) ReusedExchange [Reuses operator id: 21] +(27) ReusedExchange [Reuses operator id: 19] Output [1]: [d_date_sk#26] -(30) CometBroadcastHashJoin +(28) 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 -(31) CometProject +(29) 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] -(32) CometHashAggregate +(30) CometHashAggregate Input [3]: [brand_id#27, class_id#28, category_id#29] Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] -(33) ColumnarToRow [codegen id : 1] +(31) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(34) Exchange -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 : 3] +(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#27, class_id#28, category_id#29] -(36) Scan parquet spark_catalog.default.web_sales +(33) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] @@ -303,669 +289,626 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(37) CometFilter +(34) CometFilter Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] Condition : isnotnull(ws_item_sk#30) -(38) ReusedExchange [Reuses operator id: 15] +(35) ReusedExchange [Reuses operator id: 13] Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -(39) CometBroadcastHashJoin +(36) 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) CometProject +(37) 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) ReusedExchange [Reuses operator id: 21] +(38) ReusedExchange [Reuses operator id: 19] Output [1]: [d_date_sk#37] -(42) CometBroadcastHashJoin +(39) 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) CometProject +(40) 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) ColumnarToRow [codegen id : 2] -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(45) BroadcastExchange +(41) CometBroadcastExchange 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] +Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] -(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 +(42) CometBroadcastHashJoin +Left output [3]: [brand_id#27, class_id#28, category_id#29] +Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [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)], [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)], LeftSemi, BuildRight -(47) BroadcastExchange +(43) CometBroadcastExchange 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] +Arguments: [brand_id#27, class_id#28, category_id#29] -(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#27, class_id#28, category_id#29] -Join type: Inner -Join condition: None +(44) CometBroadcastHashJoin +Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Right output [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight -(49) Project [codegen id : 4] -Output [1]: [i_item_sk#6 AS ss_item_sk#38] +(45) CometProject 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] +Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] -(50) BroadcastExchange +(46) CometBroadcastExchange Input [1]: [ss_item_sk#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: [ss_item_sk#38] -(51) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#38] -Join type: LeftSemi -Join condition: None +(47) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Right output [1]: [ss_item_sk#38] +Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight -(52) Scan parquet spark_catalog.default.item +(48) Scan parquet spark_catalog.default.item 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 +(49) CometFilter 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 : 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] +(50) ReusedExchange [Reuses operator id: 46] Output [1]: [ss_item_sk#38] -(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 +(51) CometBroadcastHashJoin +Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Right output [1]: [ss_item_sk#38] +Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight -(57) BroadcastExchange +(52) CometBroadcastExchange 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] +Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -(58) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#39] -Join type: Inner -Join condition: None +(53) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight -(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] +(54) CometProject 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] +Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -(60) ReusedExchange [Reuses operator id: 155] -Output [1]: [d_date_sk#43] +(55) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#43, d_year#44, d_moy#45] +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 + +(56) CometFilter +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) + +(57) CometProject +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Arguments: [d_date_sk#43], [d_date_sk#43] + +(58) CometBroadcastExchange +Input [1]: [d_date_sk#43] +Arguments: [d_date_sk#43] -(61) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#43] -Join type: Inner -Join condition: None +(59) CometBroadcastHashJoin +Left 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] +Right output [1]: [d_date_sk#43] +Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight -(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] +(60) CometProject 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] +Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -(63) HashAggregate [codegen id : 11] +(61) CometHashAggregate 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#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#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] +(62) CometColumnarExchange +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] +Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(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] +(63) CometHashAggregate +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] 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))#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 : 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)))) +(64) CometFilter +Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] +Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) -(67) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] +(65) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_sold_date_sk#60 IN dynamicpruning#61)] +PartitionFilters: [isnotnull(cs_sold_date_sk#57), dynamicpruningexpression(cs_sold_date_sk#57 IN dynamicpruning#58)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(68) CometFilter -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 : 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#62] - -(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#63, i_brand_id#64, i_class_id#65, i_category_id#66] - -(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 : 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: 155] -Output [1]: [d_date_sk#67] - -(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 : 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 : 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#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 : 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 : 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#79, ws_quantity#80, ws_list_price#81, ws_sold_date_sk#82] +(66) CometFilter +Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] +Condition : isnotnull(cs_item_sk#54) + +(67) ReusedExchange [Reuses operator id: 46] +Output [1]: [ss_item_sk#59] + +(68) CometBroadcastHashJoin +Left output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] +Right output [1]: [ss_item_sk#59] +Arguments: [cs_item_sk#54], [ss_item_sk#59], LeftSemi, BuildRight + +(69) ReusedExchange [Reuses operator id: 52] +Output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] + +(70) CometBroadcastHashJoin +Left output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] +Right output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] +Arguments: [cs_item_sk#54], [i_item_sk#60], Inner, BuildRight + +(71) CometProject +Input [8]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] +Arguments: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63], [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63] + +(72) ReusedExchange [Reuses operator id: 58] +Output [1]: [d_date_sk#64] + +(73) CometBroadcastHashJoin +Left output [6]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63] +Right output [1]: [d_date_sk#64] +Arguments: [cs_sold_date_sk#57], [d_date_sk#64], Inner, BuildRight + +(74) CometProject +Input [7]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63, d_date_sk#64] +Arguments: [cs_quantity#55, cs_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63], [cs_quantity#55, cs_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] + +(75) CometHashAggregate +Input [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] +Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] +Functions [2]: [partial_sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), partial_count(1)] + +(76) CometColumnarExchange +Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#65, isEmpty#66, count#67] +Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(77) CometHashAggregate +Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#65, isEmpty#66, count#67] +Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] +Functions [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), count(1)] + +(78) CometFilter +Input [6]: [channel#68, i_brand_id#61, i_class_id#62, i_category_id#63, sales#69, number_sales#70] +Condition : (isnotnull(sales#69) AND (cast(sales#69 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) + +(79) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#82), dynamicpruningexpression(ws_sold_date_sk#82 IN dynamicpruning#83)] +PartitionFilters: [isnotnull(ws_sold_date_sk#74), dynamicpruningexpression(ws_sold_date_sk#74 IN dynamicpruning#75)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(83) CometFilter -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 : 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#84] - -(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#85, i_brand_id#86, i_class_id#87, i_category_id#88] - -(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 : 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: 155] -Output [1]: [d_date_sk#89] - -(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 : 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 : 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#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 : 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 : 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 : 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#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 : 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#52, i_brand_id#111, i_class_id#112, i_category_id#113, sum#104, isEmpty#105, sum#106] - -(102) HashAggregate [codegen id : 76] -Input [7]: [channel#52, i_brand_id#111, i_class_id#112, i_category_id#113, sum#104, isEmpty#105, sum#106] -Keys [4]: [channel#52, i_brand_id#111, i_class_id#112, i_category_id#113] -Functions [2]: [sum(sales#53), sum(number_sales#54)] -Aggregate Attributes [2]: [sum(sales#53)#107, sum(number_sales#54)#108] -Results [5]: [channel#52, i_brand_id#111, i_class_id#112, sum(sales#53)#107 AS sum_sales#114, sum(number_sales#54)#108 AS number_sales#115] - -(103) HashAggregate [codegen id : 76] -Input [5]: [channel#52, i_brand_id#111, i_class_id#112, sum_sales#114, number_sales#115] -Keys [3]: [channel#52, i_brand_id#111, i_class_id#112] -Functions [2]: [partial_sum(sum_sales#114), partial_sum(number_sales#115)] -Aggregate Attributes [3]: [sum#116, isEmpty#117, sum#118] -Results [6]: [channel#52, i_brand_id#111, i_class_id#112, sum#119, isEmpty#120, sum#121] - -(104) Exchange -Input [6]: [channel#52, i_brand_id#111, i_class_id#112, sum#119, isEmpty#120, sum#121] -Arguments: hashpartitioning(channel#52, i_brand_id#111, i_class_id#112, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(105) HashAggregate [codegen id : 77] -Input [6]: [channel#52, i_brand_id#111, i_class_id#112, sum#119, isEmpty#120, sum#121] -Keys [3]: [channel#52, i_brand_id#111, i_class_id#112] -Functions [2]: [sum(sum_sales#114), sum(number_sales#115)] -Aggregate Attributes [2]: [sum(sum_sales#114)#122, sum(number_sales#115)#123] -Results [6]: [channel#52, i_brand_id#111, i_class_id#112, null AS i_category_id#124, sum(sum_sales#114)#122 AS sum(sum_sales)#125, sum(number_sales#115)#123 AS sum(number_sales)#126] - -(106) ReusedExchange [Reuses operator id: 99] -Output [7]: [channel#52, i_brand_id#127, i_class_id#128, i_category_id#129, sum#104, isEmpty#105, sum#106] - -(107) HashAggregate [codegen id : 115] -Input [7]: [channel#52, i_brand_id#127, i_class_id#128, i_category_id#129, sum#104, isEmpty#105, sum#106] -Keys [4]: [channel#52, i_brand_id#127, i_class_id#128, i_category_id#129] -Functions [2]: [sum(sales#53), sum(number_sales#54)] -Aggregate Attributes [2]: [sum(sales#53)#107, sum(number_sales#54)#108] -Results [4]: [channel#52, i_brand_id#127, sum(sales#53)#107 AS sum_sales#130, sum(number_sales#54)#108 AS number_sales#131] - -(108) HashAggregate [codegen id : 115] -Input [4]: [channel#52, i_brand_id#127, sum_sales#130, number_sales#131] -Keys [2]: [channel#52, i_brand_id#127] -Functions [2]: [partial_sum(sum_sales#130), partial_sum(number_sales#131)] -Aggregate Attributes [3]: [sum#132, isEmpty#133, sum#134] -Results [5]: [channel#52, i_brand_id#127, sum#135, isEmpty#136, sum#137] - -(109) Exchange -Input [5]: [channel#52, i_brand_id#127, sum#135, isEmpty#136, sum#137] -Arguments: hashpartitioning(channel#52, i_brand_id#127, 5), ENSURE_REQUIREMENTS, [plan_id=11] - -(110) HashAggregate [codegen id : 116] -Input [5]: [channel#52, i_brand_id#127, sum#135, isEmpty#136, sum#137] -Keys [2]: [channel#52, i_brand_id#127] -Functions [2]: [sum(sum_sales#130), sum(number_sales#131)] -Aggregate Attributes [2]: [sum(sum_sales#130)#138, sum(number_sales#131)#139] -Results [6]: [channel#52, i_brand_id#127, null AS i_class_id#140, null AS i_category_id#141, sum(sum_sales#130)#138 AS sum(sum_sales)#142, sum(number_sales#131)#139 AS sum(number_sales)#143] - -(111) ReusedExchange [Reuses operator id: 99] -Output [7]: [channel#52, i_brand_id#144, i_class_id#145, i_category_id#146, sum#104, isEmpty#105, sum#106] - -(112) HashAggregate [codegen id : 154] -Input [7]: [channel#52, i_brand_id#144, i_class_id#145, i_category_id#146, sum#104, isEmpty#105, sum#106] -Keys [4]: [channel#52, i_brand_id#144, i_class_id#145, i_category_id#146] -Functions [2]: [sum(sales#53), sum(number_sales#54)] -Aggregate Attributes [2]: [sum(sales#53)#107, sum(number_sales#54)#108] -Results [3]: [channel#52, sum(sales#53)#107 AS sum_sales#147, sum(number_sales#54)#108 AS number_sales#148] - -(113) HashAggregate [codegen id : 154] -Input [3]: [channel#52, sum_sales#147, number_sales#148] -Keys [1]: [channel#52] -Functions [2]: [partial_sum(sum_sales#147), partial_sum(number_sales#148)] -Aggregate Attributes [3]: [sum#149, isEmpty#150, sum#151] -Results [4]: [channel#52, sum#152, isEmpty#153, sum#154] - -(114) Exchange -Input [4]: [channel#52, sum#152, isEmpty#153, sum#154] -Arguments: hashpartitioning(channel#52, 5), ENSURE_REQUIREMENTS, [plan_id=12] - -(115) HashAggregate [codegen id : 155] -Input [4]: [channel#52, sum#152, isEmpty#153, sum#154] -Keys [1]: [channel#52] -Functions [2]: [sum(sum_sales#147), sum(number_sales#148)] -Aggregate Attributes [2]: [sum(sum_sales#147)#155, sum(number_sales#148)#156] -Results [6]: [channel#52, null AS i_brand_id#157, null AS i_class_id#158, null AS i_category_id#159, sum(sum_sales#147)#155 AS sum(sum_sales)#160, sum(number_sales#148)#156 AS sum(number_sales)#161] - -(116) ReusedExchange [Reuses operator id: 99] -Output [7]: [channel#52, i_brand_id#162, i_class_id#163, i_category_id#164, sum#104, isEmpty#105, sum#106] - -(117) HashAggregate [codegen id : 193] -Input [7]: [channel#52, i_brand_id#162, i_class_id#163, i_category_id#164, sum#104, isEmpty#105, sum#106] -Keys [4]: [channel#52, i_brand_id#162, i_class_id#163, i_category_id#164] -Functions [2]: [sum(sales#53), sum(number_sales#54)] -Aggregate Attributes [2]: [sum(sales#53)#107, sum(number_sales#54)#108] -Results [2]: [sum(sales#53)#107 AS sum_sales#165, sum(number_sales#54)#108 AS number_sales#166] - -(118) HashAggregate [codegen id : 193] -Input [2]: [sum_sales#165, number_sales#166] +(80) CometFilter +Input [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] +Condition : isnotnull(ws_item_sk#71) + +(81) ReusedExchange [Reuses operator id: 46] +Output [1]: [ss_item_sk#76] + +(82) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] +Right output [1]: [ss_item_sk#76] +Arguments: [ws_item_sk#71], [ss_item_sk#76], LeftSemi, BuildRight + +(83) ReusedExchange [Reuses operator id: 52] +Output [4]: [i_item_sk#77, i_brand_id#78, i_class_id#79, i_category_id#80] + +(84) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] +Right output [4]: [i_item_sk#77, i_brand_id#78, i_class_id#79, i_category_id#80] +Arguments: [ws_item_sk#71], [i_item_sk#77], Inner, BuildRight + +(85) CometProject +Input [8]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_item_sk#77, i_brand_id#78, i_class_id#79, i_category_id#80] +Arguments: [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80], [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80] + +(86) ReusedExchange [Reuses operator id: 58] +Output [1]: [d_date_sk#81] + +(87) CometBroadcastHashJoin +Left output [6]: [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80] +Right output [1]: [d_date_sk#81] +Arguments: [ws_sold_date_sk#74], [d_date_sk#81], Inner, BuildRight + +(88) CometProject +Input [7]: [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80, d_date_sk#81] +Arguments: [ws_quantity#72, ws_list_price#73, i_brand_id#78, i_class_id#79, i_category_id#80], [ws_quantity#72, ws_list_price#73, i_brand_id#78, i_class_id#79, i_category_id#80] + +(89) CometHashAggregate +Input [5]: [ws_quantity#72, ws_list_price#73, i_brand_id#78, i_class_id#79, i_category_id#80] +Keys [3]: [i_brand_id#78, i_class_id#79, i_category_id#80] +Functions [2]: [partial_sum((cast(ws_quantity#72 as decimal(10,0)) * ws_list_price#73)), partial_count(1)] + +(90) CometColumnarExchange +Input [6]: [i_brand_id#78, i_class_id#79, i_category_id#80, sum#82, isEmpty#83, count#84] +Arguments: hashpartitioning(i_brand_id#78, i_class_id#79, i_category_id#80, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(91) CometHashAggregate +Input [6]: [i_brand_id#78, i_class_id#79, i_category_id#80, sum#82, isEmpty#83, count#84] +Keys [3]: [i_brand_id#78, i_class_id#79, i_category_id#80] +Functions [2]: [sum((cast(ws_quantity#72 as decimal(10,0)) * ws_list_price#73)), count(1)] + +(92) CometFilter +Input [6]: [channel#85, i_brand_id#78, i_class_id#79, i_category_id#80, sales#86, number_sales#87] +Condition : (isnotnull(sales#86) AND (cast(sales#86 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) + +(93) CometUnion +Child 0 Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] +Child 1 Input [6]: [channel#68, i_brand_id#61, i_class_id#62, i_category_id#63, sales#69, number_sales#70] +Child 2 Input [6]: [channel#85, i_brand_id#78, i_class_id#79, i_category_id#80, sales#86, number_sales#87] + +(94) CometHashAggregate +Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] +Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] +Functions [2]: [partial_sum(sales#50), partial_sum(number_sales#51)] + +(95) CometColumnarExchange +Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#88, isEmpty#89, sum#90] +Arguments: hashpartitioning(channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(96) CometHashAggregate +Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#88, isEmpty#89, sum#90] +Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] +Functions [2]: [sum(sales#50), sum(number_sales#51)] + +(97) ReusedExchange [Reuses operator id: 95] +Output [7]: [channel#49, i_brand_id#91, i_class_id#92, i_category_id#93, sum#88, isEmpty#89, sum#90] + +(98) CometHashAggregate +Input [7]: [channel#49, i_brand_id#91, i_class_id#92, i_category_id#93, sum#88, isEmpty#89, sum#90] +Keys [4]: [channel#49, i_brand_id#91, i_class_id#92, i_category_id#93] +Functions [2]: [sum(sales#50), sum(number_sales#51)] + +(99) CometHashAggregate +Input [5]: [channel#49, i_brand_id#91, i_class_id#92, sum_sales#94, number_sales#95] +Keys [3]: [channel#49, i_brand_id#91, i_class_id#92] +Functions [2]: [partial_sum(sum_sales#94), partial_sum(number_sales#95)] + +(100) CometColumnarExchange +Input [6]: [channel#49, i_brand_id#91, i_class_id#92, sum#96, isEmpty#97, sum#98] +Arguments: hashpartitioning(channel#49, i_brand_id#91, i_class_id#92, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(101) CometHashAggregate +Input [6]: [channel#49, i_brand_id#91, i_class_id#92, sum#96, isEmpty#97, sum#98] +Keys [3]: [channel#49, i_brand_id#91, i_class_id#92] +Functions [2]: [sum(sum_sales#94), sum(number_sales#95)] + +(102) ReusedExchange [Reuses operator id: 95] +Output [7]: [channel#49, i_brand_id#99, i_class_id#100, i_category_id#101, sum#88, isEmpty#89, sum#90] + +(103) CometHashAggregate +Input [7]: [channel#49, i_brand_id#99, i_class_id#100, i_category_id#101, sum#88, isEmpty#89, sum#90] +Keys [4]: [channel#49, i_brand_id#99, i_class_id#100, i_category_id#101] +Functions [2]: [sum(sales#50), sum(number_sales#51)] + +(104) CometHashAggregate +Input [4]: [channel#49, i_brand_id#99, sum_sales#102, number_sales#103] +Keys [2]: [channel#49, i_brand_id#99] +Functions [2]: [partial_sum(sum_sales#102), partial_sum(number_sales#103)] + +(105) CometColumnarExchange +Input [5]: [channel#49, i_brand_id#99, sum#104, isEmpty#105, sum#106] +Arguments: hashpartitioning(channel#49, i_brand_id#99, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(106) CometHashAggregate +Input [5]: [channel#49, i_brand_id#99, sum#104, isEmpty#105, sum#106] +Keys [2]: [channel#49, i_brand_id#99] +Functions [2]: [sum(sum_sales#102), sum(number_sales#103)] + +(107) ReusedExchange [Reuses operator id: 95] +Output [7]: [channel#49, i_brand_id#107, i_class_id#108, i_category_id#109, sum#88, isEmpty#89, sum#90] + +(108) CometHashAggregate +Input [7]: [channel#49, i_brand_id#107, i_class_id#108, i_category_id#109, sum#88, isEmpty#89, sum#90] +Keys [4]: [channel#49, i_brand_id#107, i_class_id#108, i_category_id#109] +Functions [2]: [sum(sales#50), sum(number_sales#51)] + +(109) CometHashAggregate +Input [3]: [channel#49, sum_sales#110, number_sales#111] +Keys [1]: [channel#49] +Functions [2]: [partial_sum(sum_sales#110), partial_sum(number_sales#111)] + +(110) CometColumnarExchange +Input [4]: [channel#49, sum#112, isEmpty#113, sum#114] +Arguments: hashpartitioning(channel#49, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(111) CometHashAggregate +Input [4]: [channel#49, sum#112, isEmpty#113, sum#114] +Keys [1]: [channel#49] +Functions [2]: [sum(sum_sales#110), sum(number_sales#111)] + +(112) ReusedExchange [Reuses operator id: 95] +Output [7]: [channel#49, i_brand_id#115, i_class_id#116, i_category_id#117, sum#88, isEmpty#89, sum#90] + +(113) CometHashAggregate +Input [7]: [channel#49, i_brand_id#115, i_class_id#116, i_category_id#117, sum#88, isEmpty#89, sum#90] +Keys [4]: [channel#49, i_brand_id#115, i_class_id#116, i_category_id#117] +Functions [2]: [sum(sales#50), sum(number_sales#51)] + +(114) CometHashAggregate +Input [2]: [sum_sales#118, number_sales#119] Keys: [] -Functions [2]: [partial_sum(sum_sales#165), partial_sum(number_sales#166)] -Aggregate Attributes [3]: [sum#167, isEmpty#168, sum#169] -Results [3]: [sum#170, isEmpty#171, sum#172] +Functions [2]: [partial_sum(sum_sales#118), partial_sum(number_sales#119)] -(119) Exchange -Input [3]: [sum#170, isEmpty#171, sum#172] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] +(115) CometColumnarExchange +Input [3]: [sum#120, isEmpty#121, sum#122] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(120) HashAggregate [codegen id : 194] -Input [3]: [sum#170, isEmpty#171, sum#172] +(116) CometHashAggregate +Input [3]: [sum#120, isEmpty#121, sum#122] Keys: [] -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]: [null AS channel#175, null AS i_brand_id#176, null AS i_class_id#177, null AS i_category_id#178, sum(sum_sales#165)#173 AS sum(sum_sales)#179, sum(number_sales#166)#174 AS sum(number_sales)#180] - -(121) Union - -(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 [2]: [sum(sum_sales#118), sum(number_sales#119)] + +(117) CometUnion +Child 0 Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] +Child 1 Input [6]: [channel#49, i_brand_id#91, i_class_id#92, i_category_id#125, sum(sum_sales)#126, sum(number_sales)#127] +Child 2 Input [6]: [channel#49, i_brand_id#99, i_class_id#128, i_category_id#129, sum(sum_sales)#130, sum(number_sales)#131] +Child 3 Input [6]: [channel#49, i_brand_id#132, i_class_id#133, i_category_id#134, sum(sum_sales)#135, sum(number_sales)#136] +Child 4 Input [6]: [channel#137, i_brand_id#138, i_class_id#139, i_category_id#140, sum(sum_sales)#141, sum(number_sales)#142] + +(118) CometHashAggregate +Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] +Keys [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] Functions: [] -Aggregate Attributes: [] -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#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] +(119) CometColumnarExchange +Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] +Arguments: hashpartitioning(channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(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] +(120) CometHashAggregate +Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] +Keys [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] Functions: [] -Aggregate Attributes: [] -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#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] +(121) CometTakeOrderedAndProject +Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#49 ASC NULLS FIRST,i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#49,i_brand_id#40,i_class_id#41,i_category_id#42,sum_sales#123,number_sales#124]), [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124], 100, [channel#49 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#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] + +(122) ColumnarToRow [codegen id : 1] +Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] ===== Subqueries ===== -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#181, ss_list_price#182, ss_sold_date_sk#183] +Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#52, [id=#53] +* ColumnarToRow (142) ++- CometHashAggregate (141) + +- CometColumnarExchange (140) + +- CometHashAggregate (139) + +- CometUnion (138) + :- CometProject (126) + : +- CometBroadcastHashJoin (125) + : :- CometScan parquet spark_catalog.default.store_sales (123) + : +- ReusedExchange (124) + :- CometProject (133) + : +- CometBroadcastHashJoin (132) + : :- CometScan parquet spark_catalog.default.catalog_sales (127) + : +- CometBroadcastExchange (131) + : +- CometProject (130) + : +- CometFilter (129) + : +- CometScan parquet spark_catalog.default.date_dim (128) + +- CometProject (137) + +- CometBroadcastHashJoin (136) + :- CometScan parquet spark_catalog.default.web_sales (134) + +- ReusedExchange (135) + + +(123) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_quantity#143, ss_list_price#144, ss_sold_date_sk#145] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#183), dynamicpruningexpression(ss_sold_date_sk#183 IN dynamicpruning#184)] +PartitionFilters: [isnotnull(ss_sold_date_sk#145), dynamicpruningexpression(ss_sold_date_sk#145 IN dynamicpruning#146)] ReadSchema: struct -(127) ReusedExchange [Reuses operator id: 21] -Output [1]: [d_date_sk#185] +(124) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#147] -(128) CometBroadcastHashJoin -Left output [3]: [ss_quantity#181, ss_list_price#182, ss_sold_date_sk#183] -Right output [1]: [d_date_sk#185] -Arguments: [ss_sold_date_sk#183], [d_date_sk#185], Inner, BuildRight +(125) CometBroadcastHashJoin +Left output [3]: [ss_quantity#143, ss_list_price#144, ss_sold_date_sk#145] +Right output [1]: [d_date_sk#147] +Arguments: [ss_sold_date_sk#145], [d_date_sk#147], Inner, BuildRight -(129) CometProject -Input [4]: [ss_quantity#181, ss_list_price#182, ss_sold_date_sk#183, d_date_sk#185] -Arguments: [quantity#186, list_price#187], [ss_quantity#181 AS quantity#186, ss_list_price#182 AS list_price#187] +(126) CometProject +Input [4]: [ss_quantity#143, ss_list_price#144, ss_sold_date_sk#145, d_date_sk#147] +Arguments: [quantity#148, list_price#149], [ss_quantity#143 AS quantity#148, ss_list_price#144 AS list_price#149] -(130) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#188, cs_list_price#189, cs_sold_date_sk#190] +(127) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#150, cs_list_price#151, cs_sold_date_sk#152] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#190), dynamicpruningexpression(cs_sold_date_sk#190 IN dynamicpruning#191)] +PartitionFilters: [isnotnull(cs_sold_date_sk#152), dynamicpruningexpression(cs_sold_date_sk#152 IN dynamicpruning#153)] ReadSchema: struct -(131) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#192, d_year#193] +(128) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#154, d_year#155] 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#192, d_year#193] -Condition : (((isnotnull(d_year#193) AND (d_year#193 >= 1998)) AND (d_year#193 <= 2000)) AND isnotnull(d_date_sk#192)) +(129) CometFilter +Input [2]: [d_date_sk#154, d_year#155] +Condition : (((isnotnull(d_year#155) AND (d_year#155 >= 1998)) AND (d_year#155 <= 2000)) AND isnotnull(d_date_sk#154)) -(133) CometProject -Input [2]: [d_date_sk#192, d_year#193] -Arguments: [d_date_sk#192], [d_date_sk#192] +(130) CometProject +Input [2]: [d_date_sk#154, d_year#155] +Arguments: [d_date_sk#154], [d_date_sk#154] -(134) CometBroadcastExchange -Input [1]: [d_date_sk#192] -Arguments: [d_date_sk#192] +(131) CometBroadcastExchange +Input [1]: [d_date_sk#154] +Arguments: [d_date_sk#154] -(135) CometBroadcastHashJoin -Left output [3]: [cs_quantity#188, cs_list_price#189, cs_sold_date_sk#190] -Right output [1]: [d_date_sk#192] -Arguments: [cs_sold_date_sk#190], [d_date_sk#192], Inner, BuildRight +(132) CometBroadcastHashJoin +Left output [3]: [cs_quantity#150, cs_list_price#151, cs_sold_date_sk#152] +Right output [1]: [d_date_sk#154] +Arguments: [cs_sold_date_sk#152], [d_date_sk#154], Inner, BuildRight -(136) CometProject -Input [4]: [cs_quantity#188, cs_list_price#189, cs_sold_date_sk#190, d_date_sk#192] -Arguments: [quantity#194, list_price#195], [cs_quantity#188 AS quantity#194, cs_list_price#189 AS list_price#195] +(133) CometProject +Input [4]: [cs_quantity#150, cs_list_price#151, cs_sold_date_sk#152, d_date_sk#154] +Arguments: [quantity#156, list_price#157], [cs_quantity#150 AS quantity#156, cs_list_price#151 AS list_price#157] -(137) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#196, ws_list_price#197, ws_sold_date_sk#198] +(134) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#158, ws_list_price#159, ws_sold_date_sk#160] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#198), dynamicpruningexpression(ws_sold_date_sk#198 IN dynamicpruning#199)] +PartitionFilters: [isnotnull(ws_sold_date_sk#160), dynamicpruningexpression(ws_sold_date_sk#160 IN dynamicpruning#161)] ReadSchema: struct -(138) ReusedExchange [Reuses operator id: 134] -Output [1]: [d_date_sk#200] +(135) ReusedExchange [Reuses operator id: 131] +Output [1]: [d_date_sk#162] -(139) CometBroadcastHashJoin -Left output [3]: [ws_quantity#196, ws_list_price#197, ws_sold_date_sk#198] -Right output [1]: [d_date_sk#200] -Arguments: [ws_sold_date_sk#198], [d_date_sk#200], Inner, BuildRight +(136) CometBroadcastHashJoin +Left output [3]: [ws_quantity#158, ws_list_price#159, ws_sold_date_sk#160] +Right output [1]: [d_date_sk#162] +Arguments: [ws_sold_date_sk#160], [d_date_sk#162], Inner, BuildRight -(140) CometProject -Input [4]: [ws_quantity#196, ws_list_price#197, ws_sold_date_sk#198, d_date_sk#200] -Arguments: [quantity#201, list_price#202], [ws_quantity#196 AS quantity#201, ws_list_price#197 AS list_price#202] +(137) CometProject +Input [4]: [ws_quantity#158, ws_list_price#159, ws_sold_date_sk#160, d_date_sk#162] +Arguments: [quantity#163, list_price#164], [ws_quantity#158 AS quantity#163, ws_list_price#159 AS list_price#164] -(141) CometUnion -Child 0 Input [2]: [quantity#186, list_price#187] -Child 1 Input [2]: [quantity#194, list_price#195] -Child 2 Input [2]: [quantity#201, list_price#202] +(138) CometUnion +Child 0 Input [2]: [quantity#148, list_price#149] +Child 1 Input [2]: [quantity#156, list_price#157] +Child 2 Input [2]: [quantity#163, list_price#164] -(142) CometHashAggregate -Input [2]: [quantity#186, list_price#187] +(139) CometHashAggregate +Input [2]: [quantity#148, list_price#149] Keys: [] -Functions [1]: [partial_avg((cast(quantity#186 as decimal(10,0)) * list_price#187))] - -(143) ColumnarToRow [codegen id : 1] -Input [2]: [sum#203, count#204] +Functions [1]: [partial_avg((cast(quantity#148 as decimal(10,0)) * list_price#149))] -(144) Exchange -Input [2]: [sum#203, count#204] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=15] +(140) CometColumnarExchange +Input [2]: [sum#165, count#166] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(145) HashAggregate [codegen id : 2] -Input [2]: [sum#203, count#204] +(141) CometHashAggregate +Input [2]: [sum#165, count#166] Keys: [] -Functions [1]: [avg((cast(quantity#186 as decimal(10,0)) * list_price#187))] -Aggregate Attributes [1]: [avg((cast(quantity#186 as decimal(10,0)) * list_price#187))#205] -Results [1]: [avg((cast(quantity#186 as decimal(10,0)) * list_price#187))#205 AS average_sales#206] +Functions [1]: [avg((cast(quantity#148 as decimal(10,0)) * list_price#149))] + +(142) ColumnarToRow [codegen id : 1] +Input [1]: [average_sales#167] -Subquery:2 Hosting operator id = 126 Hosting Expression = ss_sold_date_sk#183 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 123 Hosting Expression = ss_sold_date_sk#145 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 130 Hosting Expression = cs_sold_date_sk#190 IN dynamicpruning#191 -BroadcastExchange (150) -+- * ColumnarToRow (149) - +- CometProject (148) - +- CometFilter (147) - +- CometScan parquet spark_catalog.default.date_dim (146) +Subquery:3 Hosting operator id = 127 Hosting Expression = cs_sold_date_sk#152 IN dynamicpruning#153 +BroadcastExchange (147) ++- * ColumnarToRow (146) + +- CometProject (145) + +- CometFilter (144) + +- CometScan parquet spark_catalog.default.date_dim (143) -(146) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#192, d_year#193] +(143) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#154, d_year#155] 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 -(147) CometFilter -Input [2]: [d_date_sk#192, d_year#193] -Condition : (((isnotnull(d_year#193) AND (d_year#193 >= 1998)) AND (d_year#193 <= 2000)) AND isnotnull(d_date_sk#192)) +(144) CometFilter +Input [2]: [d_date_sk#154, d_year#155] +Condition : (((isnotnull(d_year#155) AND (d_year#155 >= 1998)) AND (d_year#155 <= 2000)) AND isnotnull(d_date_sk#154)) -(148) CometProject -Input [2]: [d_date_sk#192, d_year#193] -Arguments: [d_date_sk#192], [d_date_sk#192] +(145) CometProject +Input [2]: [d_date_sk#154, d_year#155] +Arguments: [d_date_sk#154], [d_date_sk#154] -(149) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#192] +(146) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#154] -(150) BroadcastExchange -Input [1]: [d_date_sk#192] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] +(147) BroadcastExchange +Input [1]: [d_date_sk#154] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -Subquery:4 Hosting operator id = 137 Hosting Expression = ws_sold_date_sk#198 IN dynamicpruning#191 +Subquery:4 Hosting operator id = 134 Hosting Expression = ws_sold_date_sk#160 IN dynamicpruning#153 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (155) -+- * ColumnarToRow (154) - +- CometProject (153) - +- CometFilter (152) - +- CometScan parquet spark_catalog.default.date_dim (151) +BroadcastExchange (152) ++- * ColumnarToRow (151) + +- CometProject (150) + +- CometFilter (149) + +- CometScan parquet spark_catalog.default.date_dim (148) -(151) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#43, d_year#207, d_moy#208] +(148) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#43, d_year#44, d_moy#45] 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 -(152) CometFilter -Input [3]: [d_date_sk#43, d_year#207, d_moy#208] -Condition : ((((isnotnull(d_year#207) AND isnotnull(d_moy#208)) AND (d_year#207 = 2000)) AND (d_moy#208 = 11)) AND isnotnull(d_date_sk#43)) +(149) CometFilter +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) -(153) CometProject -Input [3]: [d_date_sk#43, d_year#207, d_moy#208] +(150) CometProject +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] Arguments: [d_date_sk#43], [d_date_sk#43] -(154) ColumnarToRow [codegen id : 1] +(151) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] -(155) BroadcastExchange +(152) BroadcastExchange Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (160) -+- * ColumnarToRow (159) - +- CometProject (158) - +- CometFilter (157) - +- CometScan parquet spark_catalog.default.date_dim (156) +Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 +BroadcastExchange (157) ++- * ColumnarToRow (156) + +- CometProject (155) + +- CometFilter (154) + +- CometScan parquet spark_catalog.default.date_dim (153) -(156) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#209] +(153) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_year#168] 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 -(157) CometFilter -Input [2]: [d_date_sk#26, d_year#209] -Condition : (((isnotnull(d_year#209) AND (d_year#209 >= 1999)) AND (d_year#209 <= 2001)) AND isnotnull(d_date_sk#26)) +(154) CometFilter +Input [2]: [d_date_sk#26, d_year#168] +Condition : (((isnotnull(d_year#168) AND (d_year#168 >= 1999)) AND (d_year#168 <= 2001)) AND isnotnull(d_date_sk#26)) -(158) CometProject -Input [2]: [d_date_sk#26, d_year#209] +(155) CometProject +Input [2]: [d_date_sk#26, d_year#168] Arguments: [d_date_sk#26], [d_date_sk#26] -(159) ColumnarToRow [codegen id : 1] +(156) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] -(160) BroadcastExchange +(157) BroadcastExchange Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] -Subquery:7 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 +Subquery:7 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#55, [id=#56] +Subquery:9 Hosting operator id = 78 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] -Subquery:10 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 65 Hosting Expression = cs_sold_date_sk#57 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#55, [id=#56] +Subquery:11 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] -Subquery:12 Hosting operator id = 82 Hosting Expression = ws_sold_date_sk#82 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#74 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/simplified.txt index 7ef901a39..01803108f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/simplified.txt @@ -1,240 +1,179 @@ -TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - 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 (195) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - InputAdapter - Union - 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 (37) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - InputAdapter - Union - WholeStageCodegen (12) - Filter [sales] - Subquery #3 - WholeStageCodegen (2) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - InputAdapter - Exchange #15 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [sum,count,quantity,list_price] - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,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_quantity,cs_list_price,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_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #17 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_quantity,ws_list_price,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 (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] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,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 #6 - 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 (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,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - CometBroadcastHashJoin [i_item_sk,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 [i_brand_id,i_class_id,i_category_id] #10 - CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [cs_item_sk,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - 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 [d_date_sk] #12 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #12 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ws_item_sk,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - ReusedExchange [d_date_sk] #12 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (9) - BroadcastHashJoin [i_item_sk,ss_item_sk] - 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 - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [d_date_sk] #4 - 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] #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] - Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - BroadcastHashJoin [cs_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 - InputAdapter - ReusedExchange [d_date_sk] #4 - 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] #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] - Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - BroadcastHashJoin [ws_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 - InputAdapter - ReusedExchange [d_date_sk] #4 - 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] #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 (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] #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] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] + CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] + CometUnion [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales,sum,isEmpty,sum,sum(sales),sum(number_salesL)] + CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id] #2 + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum,sales,number_sales] + CometUnion [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Subquery #3 + WholeStageCodegen (1) + ColumnarToRow InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - 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] #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 (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 #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 - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] + CometColumnarExchange #16 + CometHashAggregate [sum,count,quantity,list_price] + CometUnion [quantity,list_price] + CometProject [ss_quantity,ss_list_price] [quantity,list_price] + CometBroadcastHashJoin [ss_quantity,ss_list_price,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_quantity,cs_list_price,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 #17 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #18 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [ws_quantity,ws_list_price] [quantity,list_price] + CometBroadcastHashJoin [ws_quantity,ws_list_price,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] #18 + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #3 + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ss_quantity,ss_list_price] + CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk] #5 + CometProject [i_item_sk] [ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,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 [brand_id,class_id,category_id] #6 + CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + CometHashAggregate [brand_id,class_id,category_id] + CometColumnarExchange [brand_id,class_id,category_id] #7 + 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,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometBroadcastHashJoin [i_item_sk,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 [i_brand_id,i_class_id,i_category_id] #10 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + 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 [d_date_sk] #12 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #12 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #13 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + ReusedExchange [d_date_sk] #12 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] + 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] + ReusedExchange [ss_item_sk] #5 + CometBroadcastExchange [d_date_sk] #15 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + ReusedSubquery [average_sales] #3 + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1)] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #19 + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,cs_quantity,cs_list_price] + CometProject [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,ss_item_sk] + CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [ss_item_sk] #5 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 + ReusedExchange [d_date_sk] #15 + CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + ReusedSubquery [average_sales] #3 + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1)] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #20 + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ws_quantity,ws_list_price] + CometProject [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,ss_item_sk] + CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [ss_item_sk] #5 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 + ReusedExchange [d_date_sk] #15 + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum,sum(sum_sales),sum(number_salesL)] + CometColumnarExchange [channel,i_brand_id,i_class_id] #21 + CometHashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum,sum_sales,number_sales] + CometHashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales,i_category_id,sum,isEmpty,sum,sum(sales),sum(number_salesL)] + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum,sum(sum_sales),sum(number_salesL)] + CometColumnarExchange [channel,i_brand_id] #22 + CometHashAggregate [channel,i_brand_id,sum,isEmpty,sum,sum_sales,number_sales] + CometHashAggregate [channel,i_brand_id,sum_sales,number_sales,i_class_id,i_category_id,sum,isEmpty,sum,sum(sales),sum(number_salesL)] + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum,sum(sum_sales),sum(number_salesL)] + CometColumnarExchange [channel] #23 + CometHashAggregate [channel,sum,isEmpty,sum,sum_sales,number_sales] + CometHashAggregate [channel,sum_sales,number_sales,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum,sum(sales),sum(number_salesL)] + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum,sum(sum_sales),sum(number_salesL)] + CometColumnarExchange #24 + CometHashAggregate [sum,isEmpty,sum,sum_sales,number_sales] + CometHashAggregate [sum_sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum,sum(sales),sum(number_salesL)] + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/explain.txt index b0dbb65ed..144f9eea9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/explain.txt @@ -1,151 +1,147 @@ == Physical Plan == -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) +* ColumnarToRow (143) ++- CometTakeOrderedAndProject (142) + +- CometUnion (141) + :- CometHashAggregate (38) + : +- CometColumnarExchange (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) + :- CometHashAggregate (63) + : +- CometColumnarExchange (62) + : +- CometHashAggregate (61) + : +- CometProject (60) + : +- CometBroadcastHashJoin (59) + : :- CometProject (57) + : : +- CometBroadcastHashJoin (56) + : : :- CometProject (54) + : : : +- CometBroadcastHashJoin (53) + : : : :- CometProject (49) + : : : : +- CometBroadcastHashJoin (48) + : : : : :- CometProject (46) + : : : : : +- CometBroadcastHashJoin (45) + : : : : : :- CometProject (43) + : : : : : : +- CometBroadcastHashJoin (42) + : : : : : : :- CometFilter (40) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (39) + : : : : : : +- ReusedExchange (41) + : : : : : +- ReusedExchange (44) + : : : : +- ReusedExchange (47) + : : : +- CometBroadcastExchange (52) + : : : +- CometFilter (51) + : : : +- CometScan parquet spark_catalog.default.customer_address (50) + : : +- ReusedExchange (55) + : +- ReusedExchange (58) + :- CometHashAggregate (89) + : +- CometColumnarExchange (88) + : +- CometHashAggregate (87) + : +- CometProject (86) + : +- CometBroadcastHashJoin (85) + : :- CometProject (83) + : : +- CometBroadcastHashJoin (82) + : : :- CometProject (80) + : : : +- CometBroadcastHashJoin (79) + : : : :- CometProject (74) + : : : : +- CometBroadcastHashJoin (73) + : : : : :- CometProject (71) + : : : : : +- CometBroadcastHashJoin (70) + : : : : : :- CometProject (68) + : : : : : : +- CometBroadcastHashJoin (67) + : : : : : : :- CometFilter (65) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (64) + : : : : : : +- ReusedExchange (66) + : : : : : +- ReusedExchange (69) + : : : : +- ReusedExchange (72) + : : : +- CometBroadcastExchange (78) + : : : +- CometProject (77) + : : : +- CometFilter (76) + : : : +- CometScan parquet spark_catalog.default.customer_address (75) + : : +- ReusedExchange (81) + : +- ReusedExchange (84) + :- CometHashAggregate (115) + : +- CometColumnarExchange (114) + : +- CometHashAggregate (113) + : +- CometProject (112) + : +- CometBroadcastHashJoin (111) + : :- CometProject (109) + : : +- CometBroadcastHashJoin (108) + : : :- CometProject (106) + : : : +- CometBroadcastHashJoin (105) + : : : :- CometProject (100) + : : : : +- CometBroadcastHashJoin (99) + : : : : :- CometProject (97) + : : : : : +- CometBroadcastHashJoin (96) + : : : : : :- CometProject (94) + : : : : : : +- CometBroadcastHashJoin (93) + : : : : : : :- CometFilter (91) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (90) + : : : : : : +- ReusedExchange (92) + : : : : : +- ReusedExchange (95) + : : : : +- ReusedExchange (98) + : : : +- CometBroadcastExchange (104) + : : : +- CometProject (103) + : : : +- CometFilter (102) + : : : +- CometScan parquet spark_catalog.default.customer_address (101) + : : +- ReusedExchange (107) + : +- ReusedExchange (110) + +- CometHashAggregate (140) + +- CometColumnarExchange (139) + +- CometHashAggregate (138) + +- CometProject (137) + +- CometBroadcastHashJoin (136) + :- CometProject (132) + : +- CometBroadcastHashJoin (131) + : :- CometProject (129) + : : +- CometBroadcastHashJoin (128) + : : :- CometProject (126) + : : : +- CometBroadcastHashJoin (125) + : : : :- CometProject (123) + : : : : +- CometBroadcastHashJoin (122) + : : : : :- CometProject (120) + : : : : : +- CometBroadcastHashJoin (119) + : : : : : :- CometFilter (117) + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (116) + : : : : : +- ReusedExchange (118) + : : : : +- ReusedExchange (121) + : : : +- ReusedExchange (124) + : : +- ReusedExchange (127) + : +- ReusedExchange (130) + +- CometBroadcastExchange (135) + +- CometFilter (134) + +- CometScan parquet spark_catalog.default.item (133) (1) Scan parquet spark_catalog.default.catalog_sales @@ -321,532 +317,515 @@ Input [11]: [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, agg1#29, ag 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] +(37) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(38) CometHashAggregate 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#64, cs_bill_cdemo_sk#65, cs_item_sk#66, cs_quantity#67, cs_list_price#68, cs_sales_price#69, cs_coupon_amt#70, cs_net_profit#71, cs_sold_date_sk#72] +(39) Scan parquet spark_catalog.default.catalog_sales +Output [9]: [cs_bill_customer_sk#50, cs_bill_cdemo_sk#51, cs_item_sk#52, cs_quantity#53, cs_list_price#54, cs_sales_price#55, cs_coupon_amt#56, cs_net_profit#57, cs_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#72), dynamicpruningexpression(cs_sold_date_sk#72 IN dynamicpruning#73)] +PartitionFilters: [isnotnull(cs_sold_date_sk#58), dynamicpruningexpression(cs_sold_date_sk#58 IN dynamicpruning#59)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(41) CometFilter -Input [9]: [cs_bill_customer_sk#64, cs_bill_cdemo_sk#65, cs_item_sk#66, cs_quantity#67, cs_list_price#68, cs_sales_price#69, cs_coupon_amt#70, cs_net_profit#71, cs_sold_date_sk#72] -Condition : ((isnotnull(cs_bill_cdemo_sk#65) AND isnotnull(cs_bill_customer_sk#64)) AND isnotnull(cs_item_sk#66)) +(40) CometFilter +Input [9]: [cs_bill_customer_sk#50, cs_bill_cdemo_sk#51, cs_item_sk#52, cs_quantity#53, cs_list_price#54, cs_sales_price#55, cs_coupon_amt#56, cs_net_profit#57, cs_sold_date_sk#58] +Condition : ((isnotnull(cs_bill_cdemo_sk#51) AND isnotnull(cs_bill_customer_sk#50)) AND isnotnull(cs_item_sk#52)) -(42) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#74, cd_dep_count#75] +(41) ReusedExchange [Reuses operator id: 6] +Output [2]: [cd_demo_sk#60, cd_dep_count#61] -(43) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#64, cs_bill_cdemo_sk#65, cs_item_sk#66, cs_quantity#67, cs_list_price#68, cs_sales_price#69, cs_coupon_amt#70, cs_net_profit#71, cs_sold_date_sk#72] -Right output [2]: [cd_demo_sk#74, cd_dep_count#75] -Arguments: [cs_bill_cdemo_sk#65], [cd_demo_sk#74], Inner, BuildRight +(42) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#50, cs_bill_cdemo_sk#51, cs_item_sk#52, cs_quantity#53, cs_list_price#54, cs_sales_price#55, cs_coupon_amt#56, cs_net_profit#57, cs_sold_date_sk#58] +Right output [2]: [cd_demo_sk#60, cd_dep_count#61] +Arguments: [cs_bill_cdemo_sk#51], [cd_demo_sk#60], Inner, BuildRight -(44) CometProject -Input [11]: [cs_bill_customer_sk#64, cs_bill_cdemo_sk#65, cs_item_sk#66, cs_quantity#67, cs_list_price#68, cs_sales_price#69, cs_coupon_amt#70, cs_net_profit#71, cs_sold_date_sk#72, cd_demo_sk#74, cd_dep_count#75] -Arguments: [cs_bill_customer_sk#64, cs_item_sk#66, cs_quantity#67, cs_list_price#68, cs_sales_price#69, cs_coupon_amt#70, cs_net_profit#71, cs_sold_date_sk#72, cd_dep_count#75], [cs_bill_customer_sk#64, cs_item_sk#66, cs_quantity#67, cs_list_price#68, cs_sales_price#69, cs_coupon_amt#70, cs_net_profit#71, cs_sold_date_sk#72, cd_dep_count#75] +(43) CometProject +Input [11]: [cs_bill_customer_sk#50, cs_bill_cdemo_sk#51, cs_item_sk#52, cs_quantity#53, cs_list_price#54, cs_sales_price#55, cs_coupon_amt#56, cs_net_profit#57, cs_sold_date_sk#58, cd_demo_sk#60, cd_dep_count#61] +Arguments: [cs_bill_customer_sk#50, cs_item_sk#52, cs_quantity#53, cs_list_price#54, cs_sales_price#55, cs_coupon_amt#56, cs_net_profit#57, cs_sold_date_sk#58, cd_dep_count#61], [cs_bill_customer_sk#50, cs_item_sk#52, cs_quantity#53, cs_list_price#54, cs_sales_price#55, cs_coupon_amt#56, cs_net_profit#57, cs_sold_date_sk#58, cd_dep_count#61] -(45) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#76, c_current_cdemo_sk#77, c_current_addr_sk#78, c_birth_year#79] +(44) ReusedExchange [Reuses operator id: 12] +Output [4]: [c_customer_sk#62, c_current_cdemo_sk#63, c_current_addr_sk#64, c_birth_year#65] -(46) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#64, cs_item_sk#66, cs_quantity#67, cs_list_price#68, cs_sales_price#69, cs_coupon_amt#70, cs_net_profit#71, cs_sold_date_sk#72, cd_dep_count#75] -Right output [4]: [c_customer_sk#76, c_current_cdemo_sk#77, c_current_addr_sk#78, c_birth_year#79] -Arguments: [cs_bill_customer_sk#64], [c_customer_sk#76], Inner, BuildRight +(45) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#50, cs_item_sk#52, cs_quantity#53, cs_list_price#54, cs_sales_price#55, cs_coupon_amt#56, cs_net_profit#57, cs_sold_date_sk#58, cd_dep_count#61] +Right output [4]: [c_customer_sk#62, c_current_cdemo_sk#63, c_current_addr_sk#64, c_birth_year#65] +Arguments: [cs_bill_customer_sk#50], [c_customer_sk#62], Inner, BuildRight -(47) CometProject -Input [13]: [cs_bill_customer_sk#64, cs_item_sk#66, cs_quantity#67, cs_list_price#68, cs_sales_price#69, cs_coupon_amt#70, cs_net_profit#71, cs_sold_date_sk#72, cd_dep_count#75, c_customer_sk#76, c_current_cdemo_sk#77, c_current_addr_sk#78, c_birth_year#79] -Arguments: [cs_item_sk#66, cs_quantity#67, cs_list_price#68, cs_sales_price#69, cs_coupon_amt#70, cs_net_profit#71, cs_sold_date_sk#72, cd_dep_count#75, c_current_cdemo_sk#77, c_current_addr_sk#78, c_birth_year#79], [cs_item_sk#66, cs_quantity#67, cs_list_price#68, cs_sales_price#69, cs_coupon_amt#70, cs_net_profit#71, cs_sold_date_sk#72, cd_dep_count#75, c_current_cdemo_sk#77, c_current_addr_sk#78, c_birth_year#79] +(46) CometProject +Input [13]: [cs_bill_customer_sk#50, cs_item_sk#52, cs_quantity#53, cs_list_price#54, cs_sales_price#55, cs_coupon_amt#56, cs_net_profit#57, cs_sold_date_sk#58, cd_dep_count#61, c_customer_sk#62, c_current_cdemo_sk#63, c_current_addr_sk#64, c_birth_year#65] +Arguments: [cs_item_sk#52, cs_quantity#53, cs_list_price#54, cs_sales_price#55, cs_coupon_amt#56, cs_net_profit#57, cs_sold_date_sk#58, cd_dep_count#61, c_current_cdemo_sk#63, c_current_addr_sk#64, c_birth_year#65], [cs_item_sk#52, cs_quantity#53, cs_list_price#54, cs_sales_price#55, cs_coupon_amt#56, cs_net_profit#57, cs_sold_date_sk#58, cd_dep_count#61, c_current_cdemo_sk#63, c_current_addr_sk#64, c_birth_year#65] -(48) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#80] +(47) ReusedExchange [Reuses operator id: 17] +Output [1]: [cd_demo_sk#66] -(49) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#66, cs_quantity#67, cs_list_price#68, cs_sales_price#69, cs_coupon_amt#70, cs_net_profit#71, cs_sold_date_sk#72, cd_dep_count#75, c_current_cdemo_sk#77, c_current_addr_sk#78, c_birth_year#79] -Right output [1]: [cd_demo_sk#80] -Arguments: [c_current_cdemo_sk#77], [cd_demo_sk#80], Inner, BuildRight +(48) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#52, cs_quantity#53, cs_list_price#54, cs_sales_price#55, cs_coupon_amt#56, cs_net_profit#57, cs_sold_date_sk#58, cd_dep_count#61, c_current_cdemo_sk#63, c_current_addr_sk#64, c_birth_year#65] +Right output [1]: [cd_demo_sk#66] +Arguments: [c_current_cdemo_sk#63], [cd_demo_sk#66], Inner, BuildRight -(50) CometProject -Input [12]: [cs_item_sk#66, cs_quantity#67, cs_list_price#68, cs_sales_price#69, cs_coupon_amt#70, cs_net_profit#71, cs_sold_date_sk#72, cd_dep_count#75, c_current_cdemo_sk#77, c_current_addr_sk#78, c_birth_year#79, cd_demo_sk#80] -Arguments: [cs_item_sk#66, cs_quantity#67, cs_list_price#68, cs_sales_price#69, cs_coupon_amt#70, cs_net_profit#71, cs_sold_date_sk#72, cd_dep_count#75, c_current_addr_sk#78, c_birth_year#79], [cs_item_sk#66, cs_quantity#67, cs_list_price#68, cs_sales_price#69, cs_coupon_amt#70, cs_net_profit#71, cs_sold_date_sk#72, cd_dep_count#75, c_current_addr_sk#78, c_birth_year#79] +(49) CometProject +Input [12]: [cs_item_sk#52, cs_quantity#53, cs_list_price#54, cs_sales_price#55, cs_coupon_amt#56, cs_net_profit#57, cs_sold_date_sk#58, cd_dep_count#61, c_current_cdemo_sk#63, c_current_addr_sk#64, c_birth_year#65, cd_demo_sk#66] +Arguments: [cs_item_sk#52, cs_quantity#53, cs_list_price#54, cs_sales_price#55, cs_coupon_amt#56, cs_net_profit#57, cs_sold_date_sk#58, cd_dep_count#61, c_current_addr_sk#64, c_birth_year#65], [cs_item_sk#52, cs_quantity#53, cs_list_price#54, cs_sales_price#55, cs_coupon_amt#56, cs_net_profit#57, cs_sold_date_sk#58, cd_dep_count#61, c_current_addr_sk#64, c_birth_year#65] -(51) Scan parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#81, ca_state#82, ca_country#83] +(50) Scan parquet spark_catalog.default.customer_address +Output [3]: [ca_address_sk#67, ca_state#68, ca_country#69] 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 -(52) CometFilter -Input [3]: [ca_address_sk#81, ca_state#82, ca_country#83] -Condition : (ca_state#82 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#81)) - -(53) CometBroadcastExchange -Input [3]: [ca_address_sk#81, ca_state#82, ca_country#83] -Arguments: [ca_address_sk#81, ca_state#82, ca_country#83] +(51) CometFilter +Input [3]: [ca_address_sk#67, ca_state#68, ca_country#69] +Condition : (ca_state#68 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#67)) -(54) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#66, cs_quantity#67, cs_list_price#68, cs_sales_price#69, cs_coupon_amt#70, cs_net_profit#71, cs_sold_date_sk#72, cd_dep_count#75, c_current_addr_sk#78, c_birth_year#79] -Right output [3]: [ca_address_sk#81, ca_state#82, ca_country#83] -Arguments: [c_current_addr_sk#78], [ca_address_sk#81], Inner, BuildRight +(52) CometBroadcastExchange +Input [3]: [ca_address_sk#67, ca_state#68, ca_country#69] +Arguments: [ca_address_sk#67, ca_state#68, ca_country#69] -(55) CometProject -Input [13]: [cs_item_sk#66, cs_quantity#67, cs_list_price#68, cs_sales_price#69, cs_coupon_amt#70, cs_net_profit#71, cs_sold_date_sk#72, cd_dep_count#75, c_current_addr_sk#78, c_birth_year#79, ca_address_sk#81, ca_state#82, ca_country#83] -Arguments: [cs_item_sk#66, cs_quantity#67, cs_list_price#68, cs_sales_price#69, cs_coupon_amt#70, cs_net_profit#71, cs_sold_date_sk#72, cd_dep_count#75, c_birth_year#79, ca_state#82, ca_country#83], [cs_item_sk#66, cs_quantity#67, cs_list_price#68, cs_sales_price#69, cs_coupon_amt#70, cs_net_profit#71, cs_sold_date_sk#72, cd_dep_count#75, c_birth_year#79, ca_state#82, ca_country#83] +(53) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#52, cs_quantity#53, cs_list_price#54, cs_sales_price#55, cs_coupon_amt#56, cs_net_profit#57, cs_sold_date_sk#58, cd_dep_count#61, c_current_addr_sk#64, c_birth_year#65] +Right output [3]: [ca_address_sk#67, ca_state#68, ca_country#69] +Arguments: [c_current_addr_sk#64], [ca_address_sk#67], Inner, BuildRight -(56) ReusedExchange [Reuses operator id: 28] -Output [1]: [d_date_sk#84] +(54) CometProject +Input [13]: [cs_item_sk#52, cs_quantity#53, cs_list_price#54, cs_sales_price#55, cs_coupon_amt#56, cs_net_profit#57, cs_sold_date_sk#58, cd_dep_count#61, c_current_addr_sk#64, c_birth_year#65, ca_address_sk#67, ca_state#68, ca_country#69] +Arguments: [cs_item_sk#52, cs_quantity#53, cs_list_price#54, cs_sales_price#55, cs_coupon_amt#56, cs_net_profit#57, cs_sold_date_sk#58, cd_dep_count#61, c_birth_year#65, ca_state#68, ca_country#69], [cs_item_sk#52, cs_quantity#53, cs_list_price#54, cs_sales_price#55, cs_coupon_amt#56, cs_net_profit#57, cs_sold_date_sk#58, cd_dep_count#61, c_birth_year#65, ca_state#68, ca_country#69] -(57) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#66, cs_quantity#67, cs_list_price#68, cs_sales_price#69, cs_coupon_amt#70, cs_net_profit#71, cs_sold_date_sk#72, cd_dep_count#75, c_birth_year#79, ca_state#82, ca_country#83] -Right output [1]: [d_date_sk#84] -Arguments: [cs_sold_date_sk#72], [d_date_sk#84], Inner, BuildRight +(55) ReusedExchange [Reuses operator id: 28] +Output [1]: [d_date_sk#70] -(58) CometProject -Input [12]: [cs_item_sk#66, cs_quantity#67, cs_list_price#68, cs_sales_price#69, cs_coupon_amt#70, cs_net_profit#71, cs_sold_date_sk#72, cd_dep_count#75, c_birth_year#79, ca_state#82, ca_country#83, d_date_sk#84] -Arguments: [cs_item_sk#66, cs_quantity#67, cs_list_price#68, cs_sales_price#69, cs_coupon_amt#70, cs_net_profit#71, cd_dep_count#75, c_birth_year#79, ca_state#82, ca_country#83], [cs_item_sk#66, cs_quantity#67, cs_list_price#68, cs_sales_price#69, cs_coupon_amt#70, cs_net_profit#71, cd_dep_count#75, c_birth_year#79, ca_state#82, ca_country#83] +(56) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#52, cs_quantity#53, cs_list_price#54, cs_sales_price#55, cs_coupon_amt#56, cs_net_profit#57, cs_sold_date_sk#58, cd_dep_count#61, c_birth_year#65, ca_state#68, ca_country#69] +Right output [1]: [d_date_sk#70] +Arguments: [cs_sold_date_sk#58], [d_date_sk#70], Inner, BuildRight -(59) ReusedExchange [Reuses operator id: 33] -Output [2]: [i_item_sk#85, i_item_id#86] +(57) CometProject +Input [12]: [cs_item_sk#52, cs_quantity#53, cs_list_price#54, cs_sales_price#55, cs_coupon_amt#56, cs_net_profit#57, cs_sold_date_sk#58, cd_dep_count#61, c_birth_year#65, ca_state#68, ca_country#69, d_date_sk#70] +Arguments: [cs_item_sk#52, cs_quantity#53, cs_list_price#54, cs_sales_price#55, cs_coupon_amt#56, cs_net_profit#57, cd_dep_count#61, c_birth_year#65, ca_state#68, ca_country#69], [cs_item_sk#52, cs_quantity#53, cs_list_price#54, cs_sales_price#55, cs_coupon_amt#56, cs_net_profit#57, cd_dep_count#61, c_birth_year#65, ca_state#68, ca_country#69] -(60) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#66, cs_quantity#67, cs_list_price#68, cs_sales_price#69, cs_coupon_amt#70, cs_net_profit#71, cd_dep_count#75, c_birth_year#79, ca_state#82, ca_country#83] -Right output [2]: [i_item_sk#85, i_item_id#86] -Arguments: [cs_item_sk#66], [i_item_sk#85], Inner, BuildRight +(58) ReusedExchange [Reuses operator id: 33] +Output [2]: [i_item_sk#71, i_item_id#72] -(61) CometProject -Input [12]: [cs_item_sk#66, cs_quantity#67, cs_list_price#68, cs_sales_price#69, cs_coupon_amt#70, cs_net_profit#71, cd_dep_count#75, c_birth_year#79, ca_state#82, ca_country#83, i_item_sk#85, i_item_id#86] -Arguments: [i_item_id#86, ca_country#83, ca_state#82, agg1#87, agg2#88, agg3#89, agg4#90, agg5#91, agg6#92, agg7#93], [i_item_id#86, ca_country#83, ca_state#82, cast(cs_quantity#67 as decimal(12,2)) AS agg1#87, cast(cs_list_price#68 as decimal(12,2)) AS agg2#88, cast(cs_coupon_amt#70 as decimal(12,2)) AS agg3#89, cast(cs_sales_price#69 as decimal(12,2)) AS agg4#90, cast(cs_net_profit#71 as decimal(12,2)) AS agg5#91, cast(c_birth_year#79 as decimal(12,2)) AS agg6#92, cast(cd_dep_count#75 as decimal(12,2)) AS agg7#93] +(59) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#52, cs_quantity#53, cs_list_price#54, cs_sales_price#55, cs_coupon_amt#56, cs_net_profit#57, cd_dep_count#61, c_birth_year#65, ca_state#68, ca_country#69] +Right output [2]: [i_item_sk#71, i_item_id#72] +Arguments: [cs_item_sk#52], [i_item_sk#71], Inner, BuildRight -(62) CometHashAggregate -Input [10]: [i_item_id#86, ca_country#83, ca_state#82, agg1#87, agg2#88, agg3#89, agg4#90, agg5#91, agg6#92, agg7#93] -Keys [3]: [i_item_id#86, ca_country#83, ca_state#82] -Functions [7]: [partial_avg(agg1#87), partial_avg(agg2#88), partial_avg(agg3#89), partial_avg(agg4#90), partial_avg(agg5#91), partial_avg(agg6#92), partial_avg(agg7#93)] +(60) CometProject +Input [12]: [cs_item_sk#52, cs_quantity#53, cs_list_price#54, cs_sales_price#55, cs_coupon_amt#56, cs_net_profit#57, cd_dep_count#61, c_birth_year#65, ca_state#68, ca_country#69, i_item_sk#71, i_item_id#72] +Arguments: [i_item_id#72, ca_country#69, ca_state#68, agg1#73, agg2#74, agg3#75, agg4#76, agg5#77, agg6#78, agg7#79], [i_item_id#72, ca_country#69, ca_state#68, cast(cs_quantity#53 as decimal(12,2)) AS agg1#73, cast(cs_list_price#54 as decimal(12,2)) AS agg2#74, cast(cs_coupon_amt#56 as decimal(12,2)) AS agg3#75, cast(cs_sales_price#55 as decimal(12,2)) AS agg4#76, cast(cs_net_profit#57 as decimal(12,2)) AS agg5#77, cast(c_birth_year#65 as decimal(12,2)) AS agg6#78, cast(cd_dep_count#61 as decimal(12,2)) AS agg7#79] -(63) ColumnarToRow [codegen id : 3] -Input [17]: [i_item_id#86, ca_country#83, ca_state#82, sum#94, count#95, sum#96, count#97, sum#98, count#99, sum#100, count#101, sum#102, count#103, sum#104, count#105, sum#106, count#107] +(61) CometHashAggregate +Input [10]: [i_item_id#72, ca_country#69, ca_state#68, agg1#73, agg2#74, agg3#75, agg4#76, agg5#77, agg6#78, agg7#79] +Keys [3]: [i_item_id#72, ca_country#69, ca_state#68] +Functions [7]: [partial_avg(agg1#73), partial_avg(agg2#74), partial_avg(agg3#75), partial_avg(agg4#76), partial_avg(agg5#77), partial_avg(agg6#78), partial_avg(agg7#79)] -(64) Exchange -Input [17]: [i_item_id#86, ca_country#83, ca_state#82, sum#94, count#95, sum#96, count#97, sum#98, count#99, sum#100, count#101, sum#102, count#103, sum#104, count#105, sum#106, count#107] -Arguments: hashpartitioning(i_item_id#86, ca_country#83, ca_state#82, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(62) CometColumnarExchange +Input [17]: [i_item_id#72, ca_country#69, ca_state#68, sum#80, count#81, sum#82, count#83, sum#84, count#85, sum#86, count#87, sum#88, count#89, sum#90, count#91, sum#92, count#93] +Arguments: hashpartitioning(i_item_id#72, ca_country#69, ca_state#68, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(65) HashAggregate [codegen id : 4] -Input [17]: [i_item_id#86, ca_country#83, ca_state#82, sum#94, count#95, sum#96, count#97, sum#98, count#99, sum#100, count#101, sum#102, count#103, sum#104, count#105, sum#106, count#107] -Keys [3]: [i_item_id#86, ca_country#83, ca_state#82] -Functions [7]: [avg(agg1#87), avg(agg2#88), avg(agg3#89), avg(agg4#90), avg(agg5#91), avg(agg6#92), avg(agg7#93)] -Aggregate Attributes [7]: [avg(agg1#87)#108, avg(agg2#88)#109, avg(agg3#89)#110, avg(agg4#90)#111, avg(agg5#91)#112, avg(agg6#92)#113, avg(agg7#93)#114] -Results [11]: [i_item_id#86, ca_country#83, ca_state#82, null AS county#115, avg(agg1#87)#108 AS agg1#116, avg(agg2#88)#109 AS agg2#117, avg(agg3#89)#110 AS agg3#118, avg(agg4#90)#111 AS agg4#119, avg(agg5#91)#112 AS agg5#120, avg(agg6#92)#113 AS agg6#121, avg(agg7#93)#114 AS agg7#122] +(63) CometHashAggregate +Input [17]: [i_item_id#72, ca_country#69, ca_state#68, sum#80, count#81, sum#82, count#83, sum#84, count#85, sum#86, count#87, sum#88, count#89, sum#90, count#91, sum#92, count#93] +Keys [3]: [i_item_id#72, ca_country#69, ca_state#68] +Functions [7]: [avg(agg1#73), avg(agg2#74), avg(agg3#75), avg(agg4#76), avg(agg5#77), avg(agg6#78), avg(agg7#79)] -(66) Scan parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#123, cs_bill_cdemo_sk#124, cs_item_sk#125, cs_quantity#126, cs_list_price#127, cs_sales_price#128, cs_coupon_amt#129, cs_net_profit#130, cs_sold_date_sk#131] +(64) Scan parquet spark_catalog.default.catalog_sales +Output [9]: [cs_bill_customer_sk#94, cs_bill_cdemo_sk#95, cs_item_sk#96, cs_quantity#97, cs_list_price#98, cs_sales_price#99, cs_coupon_amt#100, cs_net_profit#101, cs_sold_date_sk#102] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#131), dynamicpruningexpression(cs_sold_date_sk#131 IN dynamicpruning#132)] +PartitionFilters: [isnotnull(cs_sold_date_sk#102), dynamicpruningexpression(cs_sold_date_sk#102 IN dynamicpruning#103)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(67) CometFilter -Input [9]: [cs_bill_customer_sk#123, cs_bill_cdemo_sk#124, cs_item_sk#125, cs_quantity#126, cs_list_price#127, cs_sales_price#128, cs_coupon_amt#129, cs_net_profit#130, cs_sold_date_sk#131] -Condition : ((isnotnull(cs_bill_cdemo_sk#124) AND isnotnull(cs_bill_customer_sk#123)) AND isnotnull(cs_item_sk#125)) +(65) CometFilter +Input [9]: [cs_bill_customer_sk#94, cs_bill_cdemo_sk#95, cs_item_sk#96, cs_quantity#97, cs_list_price#98, cs_sales_price#99, cs_coupon_amt#100, cs_net_profit#101, cs_sold_date_sk#102] +Condition : ((isnotnull(cs_bill_cdemo_sk#95) AND isnotnull(cs_bill_customer_sk#94)) AND isnotnull(cs_item_sk#96)) -(68) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#133, cd_dep_count#134] +(66) ReusedExchange [Reuses operator id: 6] +Output [2]: [cd_demo_sk#104, cd_dep_count#105] -(69) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#123, cs_bill_cdemo_sk#124, cs_item_sk#125, cs_quantity#126, cs_list_price#127, cs_sales_price#128, cs_coupon_amt#129, cs_net_profit#130, cs_sold_date_sk#131] -Right output [2]: [cd_demo_sk#133, cd_dep_count#134] -Arguments: [cs_bill_cdemo_sk#124], [cd_demo_sk#133], Inner, BuildRight +(67) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#94, cs_bill_cdemo_sk#95, cs_item_sk#96, cs_quantity#97, cs_list_price#98, cs_sales_price#99, cs_coupon_amt#100, cs_net_profit#101, cs_sold_date_sk#102] +Right output [2]: [cd_demo_sk#104, cd_dep_count#105] +Arguments: [cs_bill_cdemo_sk#95], [cd_demo_sk#104], Inner, BuildRight -(70) CometProject -Input [11]: [cs_bill_customer_sk#123, cs_bill_cdemo_sk#124, cs_item_sk#125, cs_quantity#126, cs_list_price#127, cs_sales_price#128, cs_coupon_amt#129, cs_net_profit#130, cs_sold_date_sk#131, cd_demo_sk#133, cd_dep_count#134] -Arguments: [cs_bill_customer_sk#123, cs_item_sk#125, cs_quantity#126, cs_list_price#127, cs_sales_price#128, cs_coupon_amt#129, cs_net_profit#130, cs_sold_date_sk#131, cd_dep_count#134], [cs_bill_customer_sk#123, cs_item_sk#125, cs_quantity#126, cs_list_price#127, cs_sales_price#128, cs_coupon_amt#129, cs_net_profit#130, cs_sold_date_sk#131, cd_dep_count#134] +(68) CometProject +Input [11]: [cs_bill_customer_sk#94, cs_bill_cdemo_sk#95, cs_item_sk#96, cs_quantity#97, cs_list_price#98, cs_sales_price#99, cs_coupon_amt#100, cs_net_profit#101, cs_sold_date_sk#102, cd_demo_sk#104, cd_dep_count#105] +Arguments: [cs_bill_customer_sk#94, cs_item_sk#96, cs_quantity#97, cs_list_price#98, cs_sales_price#99, cs_coupon_amt#100, cs_net_profit#101, cs_sold_date_sk#102, cd_dep_count#105], [cs_bill_customer_sk#94, cs_item_sk#96, cs_quantity#97, cs_list_price#98, cs_sales_price#99, cs_coupon_amt#100, cs_net_profit#101, cs_sold_date_sk#102, cd_dep_count#105] -(71) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#135, c_current_cdemo_sk#136, c_current_addr_sk#137, c_birth_year#138] +(69) ReusedExchange [Reuses operator id: 12] +Output [4]: [c_customer_sk#106, c_current_cdemo_sk#107, c_current_addr_sk#108, c_birth_year#109] -(72) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#123, cs_item_sk#125, cs_quantity#126, cs_list_price#127, cs_sales_price#128, cs_coupon_amt#129, cs_net_profit#130, cs_sold_date_sk#131, cd_dep_count#134] -Right output [4]: [c_customer_sk#135, c_current_cdemo_sk#136, c_current_addr_sk#137, c_birth_year#138] -Arguments: [cs_bill_customer_sk#123], [c_customer_sk#135], Inner, BuildRight +(70) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#94, cs_item_sk#96, cs_quantity#97, cs_list_price#98, cs_sales_price#99, cs_coupon_amt#100, cs_net_profit#101, cs_sold_date_sk#102, cd_dep_count#105] +Right output [4]: [c_customer_sk#106, c_current_cdemo_sk#107, c_current_addr_sk#108, c_birth_year#109] +Arguments: [cs_bill_customer_sk#94], [c_customer_sk#106], Inner, BuildRight -(73) CometProject -Input [13]: [cs_bill_customer_sk#123, cs_item_sk#125, cs_quantity#126, cs_list_price#127, cs_sales_price#128, cs_coupon_amt#129, cs_net_profit#130, cs_sold_date_sk#131, cd_dep_count#134, c_customer_sk#135, c_current_cdemo_sk#136, c_current_addr_sk#137, c_birth_year#138] -Arguments: [cs_item_sk#125, cs_quantity#126, cs_list_price#127, cs_sales_price#128, cs_coupon_amt#129, cs_net_profit#130, cs_sold_date_sk#131, cd_dep_count#134, c_current_cdemo_sk#136, c_current_addr_sk#137, c_birth_year#138], [cs_item_sk#125, cs_quantity#126, cs_list_price#127, cs_sales_price#128, cs_coupon_amt#129, cs_net_profit#130, cs_sold_date_sk#131, cd_dep_count#134, c_current_cdemo_sk#136, c_current_addr_sk#137, c_birth_year#138] +(71) CometProject +Input [13]: [cs_bill_customer_sk#94, cs_item_sk#96, cs_quantity#97, cs_list_price#98, cs_sales_price#99, cs_coupon_amt#100, cs_net_profit#101, cs_sold_date_sk#102, cd_dep_count#105, c_customer_sk#106, c_current_cdemo_sk#107, c_current_addr_sk#108, c_birth_year#109] +Arguments: [cs_item_sk#96, cs_quantity#97, cs_list_price#98, cs_sales_price#99, cs_coupon_amt#100, cs_net_profit#101, cs_sold_date_sk#102, cd_dep_count#105, c_current_cdemo_sk#107, c_current_addr_sk#108, c_birth_year#109], [cs_item_sk#96, cs_quantity#97, cs_list_price#98, cs_sales_price#99, cs_coupon_amt#100, cs_net_profit#101, cs_sold_date_sk#102, cd_dep_count#105, c_current_cdemo_sk#107, c_current_addr_sk#108, c_birth_year#109] -(74) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#139] +(72) ReusedExchange [Reuses operator id: 17] +Output [1]: [cd_demo_sk#110] -(75) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#125, cs_quantity#126, cs_list_price#127, cs_sales_price#128, cs_coupon_amt#129, cs_net_profit#130, cs_sold_date_sk#131, cd_dep_count#134, c_current_cdemo_sk#136, c_current_addr_sk#137, c_birth_year#138] -Right output [1]: [cd_demo_sk#139] -Arguments: [c_current_cdemo_sk#136], [cd_demo_sk#139], Inner, BuildRight +(73) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#96, cs_quantity#97, cs_list_price#98, cs_sales_price#99, cs_coupon_amt#100, cs_net_profit#101, cs_sold_date_sk#102, cd_dep_count#105, c_current_cdemo_sk#107, c_current_addr_sk#108, c_birth_year#109] +Right output [1]: [cd_demo_sk#110] +Arguments: [c_current_cdemo_sk#107], [cd_demo_sk#110], Inner, BuildRight -(76) CometProject -Input [12]: [cs_item_sk#125, cs_quantity#126, cs_list_price#127, cs_sales_price#128, cs_coupon_amt#129, cs_net_profit#130, cs_sold_date_sk#131, cd_dep_count#134, c_current_cdemo_sk#136, c_current_addr_sk#137, c_birth_year#138, cd_demo_sk#139] -Arguments: [cs_item_sk#125, cs_quantity#126, cs_list_price#127, cs_sales_price#128, cs_coupon_amt#129, cs_net_profit#130, cs_sold_date_sk#131, cd_dep_count#134, c_current_addr_sk#137, c_birth_year#138], [cs_item_sk#125, cs_quantity#126, cs_list_price#127, cs_sales_price#128, cs_coupon_amt#129, cs_net_profit#130, cs_sold_date_sk#131, cd_dep_count#134, c_current_addr_sk#137, c_birth_year#138] +(74) CometProject +Input [12]: [cs_item_sk#96, cs_quantity#97, cs_list_price#98, cs_sales_price#99, cs_coupon_amt#100, cs_net_profit#101, cs_sold_date_sk#102, cd_dep_count#105, c_current_cdemo_sk#107, c_current_addr_sk#108, c_birth_year#109, cd_demo_sk#110] +Arguments: [cs_item_sk#96, cs_quantity#97, cs_list_price#98, cs_sales_price#99, cs_coupon_amt#100, cs_net_profit#101, cs_sold_date_sk#102, cd_dep_count#105, c_current_addr_sk#108, c_birth_year#109], [cs_item_sk#96, cs_quantity#97, cs_list_price#98, cs_sales_price#99, cs_coupon_amt#100, cs_net_profit#101, cs_sold_date_sk#102, cd_dep_count#105, c_current_addr_sk#108, c_birth_year#109] -(77) Scan parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#140, ca_state#141, ca_country#142] +(75) Scan parquet spark_catalog.default.customer_address +Output [3]: [ca_address_sk#111, ca_state#112, ca_country#113] 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 -(78) CometFilter -Input [3]: [ca_address_sk#140, ca_state#141, ca_country#142] -Condition : (ca_state#141 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#140)) +(76) CometFilter +Input [3]: [ca_address_sk#111, ca_state#112, ca_country#113] +Condition : (ca_state#112 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#111)) -(79) CometProject -Input [3]: [ca_address_sk#140, ca_state#141, ca_country#142] -Arguments: [ca_address_sk#140, ca_country#142], [ca_address_sk#140, ca_country#142] +(77) CometProject +Input [3]: [ca_address_sk#111, ca_state#112, ca_country#113] +Arguments: [ca_address_sk#111, ca_country#113], [ca_address_sk#111, ca_country#113] -(80) CometBroadcastExchange -Input [2]: [ca_address_sk#140, ca_country#142] -Arguments: [ca_address_sk#140, ca_country#142] +(78) CometBroadcastExchange +Input [2]: [ca_address_sk#111, ca_country#113] +Arguments: [ca_address_sk#111, ca_country#113] -(81) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#125, cs_quantity#126, cs_list_price#127, cs_sales_price#128, cs_coupon_amt#129, cs_net_profit#130, cs_sold_date_sk#131, cd_dep_count#134, c_current_addr_sk#137, c_birth_year#138] -Right output [2]: [ca_address_sk#140, ca_country#142] -Arguments: [c_current_addr_sk#137], [ca_address_sk#140], Inner, BuildRight +(79) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#96, cs_quantity#97, cs_list_price#98, cs_sales_price#99, cs_coupon_amt#100, cs_net_profit#101, cs_sold_date_sk#102, cd_dep_count#105, c_current_addr_sk#108, c_birth_year#109] +Right output [2]: [ca_address_sk#111, ca_country#113] +Arguments: [c_current_addr_sk#108], [ca_address_sk#111], Inner, BuildRight -(82) CometProject -Input [12]: [cs_item_sk#125, cs_quantity#126, cs_list_price#127, cs_sales_price#128, cs_coupon_amt#129, cs_net_profit#130, cs_sold_date_sk#131, cd_dep_count#134, c_current_addr_sk#137, c_birth_year#138, ca_address_sk#140, ca_country#142] -Arguments: [cs_item_sk#125, cs_quantity#126, cs_list_price#127, cs_sales_price#128, cs_coupon_amt#129, cs_net_profit#130, cs_sold_date_sk#131, cd_dep_count#134, c_birth_year#138, ca_country#142], [cs_item_sk#125, cs_quantity#126, cs_list_price#127, cs_sales_price#128, cs_coupon_amt#129, cs_net_profit#130, cs_sold_date_sk#131, cd_dep_count#134, c_birth_year#138, ca_country#142] +(80) CometProject +Input [12]: [cs_item_sk#96, cs_quantity#97, cs_list_price#98, cs_sales_price#99, cs_coupon_amt#100, cs_net_profit#101, cs_sold_date_sk#102, cd_dep_count#105, c_current_addr_sk#108, c_birth_year#109, ca_address_sk#111, ca_country#113] +Arguments: [cs_item_sk#96, cs_quantity#97, cs_list_price#98, cs_sales_price#99, cs_coupon_amt#100, cs_net_profit#101, cs_sold_date_sk#102, cd_dep_count#105, c_birth_year#109, ca_country#113], [cs_item_sk#96, cs_quantity#97, cs_list_price#98, cs_sales_price#99, cs_coupon_amt#100, cs_net_profit#101, cs_sold_date_sk#102, cd_dep_count#105, c_birth_year#109, ca_country#113] -(83) ReusedExchange [Reuses operator id: 28] -Output [1]: [d_date_sk#143] +(81) ReusedExchange [Reuses operator id: 28] +Output [1]: [d_date_sk#114] -(84) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#125, cs_quantity#126, cs_list_price#127, cs_sales_price#128, cs_coupon_amt#129, cs_net_profit#130, cs_sold_date_sk#131, cd_dep_count#134, c_birth_year#138, ca_country#142] -Right output [1]: [d_date_sk#143] -Arguments: [cs_sold_date_sk#131], [d_date_sk#143], Inner, BuildRight +(82) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#96, cs_quantity#97, cs_list_price#98, cs_sales_price#99, cs_coupon_amt#100, cs_net_profit#101, cs_sold_date_sk#102, cd_dep_count#105, c_birth_year#109, ca_country#113] +Right output [1]: [d_date_sk#114] +Arguments: [cs_sold_date_sk#102], [d_date_sk#114], Inner, BuildRight -(85) CometProject -Input [11]: [cs_item_sk#125, cs_quantity#126, cs_list_price#127, cs_sales_price#128, cs_coupon_amt#129, cs_net_profit#130, cs_sold_date_sk#131, cd_dep_count#134, c_birth_year#138, ca_country#142, d_date_sk#143] -Arguments: [cs_item_sk#125, cs_quantity#126, cs_list_price#127, cs_sales_price#128, cs_coupon_amt#129, cs_net_profit#130, cd_dep_count#134, c_birth_year#138, ca_country#142], [cs_item_sk#125, cs_quantity#126, cs_list_price#127, cs_sales_price#128, cs_coupon_amt#129, cs_net_profit#130, cd_dep_count#134, c_birth_year#138, ca_country#142] +(83) CometProject +Input [11]: [cs_item_sk#96, cs_quantity#97, cs_list_price#98, cs_sales_price#99, cs_coupon_amt#100, cs_net_profit#101, cs_sold_date_sk#102, cd_dep_count#105, c_birth_year#109, ca_country#113, d_date_sk#114] +Arguments: [cs_item_sk#96, cs_quantity#97, cs_list_price#98, cs_sales_price#99, cs_coupon_amt#100, cs_net_profit#101, cd_dep_count#105, c_birth_year#109, ca_country#113], [cs_item_sk#96, cs_quantity#97, cs_list_price#98, cs_sales_price#99, cs_coupon_amt#100, cs_net_profit#101, cd_dep_count#105, c_birth_year#109, ca_country#113] -(86) ReusedExchange [Reuses operator id: 33] -Output [2]: [i_item_sk#144, i_item_id#145] +(84) ReusedExchange [Reuses operator id: 33] +Output [2]: [i_item_sk#115, i_item_id#116] -(87) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#125, cs_quantity#126, cs_list_price#127, cs_sales_price#128, cs_coupon_amt#129, cs_net_profit#130, cd_dep_count#134, c_birth_year#138, ca_country#142] -Right output [2]: [i_item_sk#144, i_item_id#145] -Arguments: [cs_item_sk#125], [i_item_sk#144], Inner, BuildRight +(85) CometBroadcastHashJoin +Left output [9]: [cs_item_sk#96, cs_quantity#97, cs_list_price#98, cs_sales_price#99, cs_coupon_amt#100, cs_net_profit#101, cd_dep_count#105, c_birth_year#109, ca_country#113] +Right output [2]: [i_item_sk#115, i_item_id#116] +Arguments: [cs_item_sk#96], [i_item_sk#115], Inner, BuildRight -(88) CometProject -Input [11]: [cs_item_sk#125, cs_quantity#126, cs_list_price#127, cs_sales_price#128, cs_coupon_amt#129, cs_net_profit#130, cd_dep_count#134, c_birth_year#138, ca_country#142, i_item_sk#144, i_item_id#145] -Arguments: [i_item_id#145, ca_country#142, agg1#146, agg2#147, agg3#148, agg4#149, agg5#150, agg6#151, agg7#152], [i_item_id#145, ca_country#142, cast(cs_quantity#126 as decimal(12,2)) AS agg1#146, cast(cs_list_price#127 as decimal(12,2)) AS agg2#147, cast(cs_coupon_amt#129 as decimal(12,2)) AS agg3#148, cast(cs_sales_price#128 as decimal(12,2)) AS agg4#149, cast(cs_net_profit#130 as decimal(12,2)) AS agg5#150, cast(c_birth_year#138 as decimal(12,2)) AS agg6#151, cast(cd_dep_count#134 as decimal(12,2)) AS agg7#152] +(86) CometProject +Input [11]: [cs_item_sk#96, cs_quantity#97, cs_list_price#98, cs_sales_price#99, cs_coupon_amt#100, cs_net_profit#101, cd_dep_count#105, c_birth_year#109, ca_country#113, i_item_sk#115, i_item_id#116] +Arguments: [i_item_id#116, ca_country#113, agg1#117, agg2#118, agg3#119, agg4#120, agg5#121, agg6#122, agg7#123], [i_item_id#116, ca_country#113, cast(cs_quantity#97 as decimal(12,2)) AS agg1#117, cast(cs_list_price#98 as decimal(12,2)) AS agg2#118, cast(cs_coupon_amt#100 as decimal(12,2)) AS agg3#119, cast(cs_sales_price#99 as decimal(12,2)) AS agg4#120, cast(cs_net_profit#101 as decimal(12,2)) AS agg5#121, cast(c_birth_year#109 as decimal(12,2)) AS agg6#122, cast(cd_dep_count#105 as decimal(12,2)) AS agg7#123] + +(87) CometHashAggregate +Input [9]: [i_item_id#116, ca_country#113, agg1#117, agg2#118, agg3#119, agg4#120, agg5#121, agg6#122, agg7#123] +Keys [2]: [i_item_id#116, ca_country#113] +Functions [7]: [partial_avg(agg1#117), partial_avg(agg2#118), partial_avg(agg3#119), partial_avg(agg4#120), partial_avg(agg5#121), partial_avg(agg6#122), partial_avg(agg7#123)] + +(88) CometColumnarExchange +Input [16]: [i_item_id#116, ca_country#113, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135, sum#136, count#137] +Arguments: hashpartitioning(i_item_id#116, ca_country#113, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (89) CometHashAggregate -Input [9]: [i_item_id#145, ca_country#142, agg1#146, agg2#147, agg3#148, agg4#149, agg5#150, agg6#151, agg7#152] -Keys [2]: [i_item_id#145, ca_country#142] -Functions [7]: [partial_avg(agg1#146), partial_avg(agg2#147), partial_avg(agg3#148), partial_avg(agg4#149), partial_avg(agg5#150), partial_avg(agg6#151), partial_avg(agg7#152)] - -(90) ColumnarToRow [codegen id : 5] -Input [16]: [i_item_id#145, ca_country#142, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160, sum#161, count#162, sum#163, count#164, sum#165, count#166] - -(91) Exchange -Input [16]: [i_item_id#145, ca_country#142, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160, sum#161, count#162, sum#163, count#164, sum#165, count#166] -Arguments: hashpartitioning(i_item_id#145, ca_country#142, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(92) HashAggregate [codegen id : 6] -Input [16]: [i_item_id#145, ca_country#142, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160, sum#161, count#162, sum#163, count#164, sum#165, count#166] -Keys [2]: [i_item_id#145, ca_country#142] -Functions [7]: [avg(agg1#146), avg(agg2#147), avg(agg3#148), avg(agg4#149), avg(agg5#150), avg(agg6#151), avg(agg7#152)] -Aggregate Attributes [7]: [avg(agg1#146)#167, avg(agg2#147)#168, avg(agg3#148)#169, avg(agg4#149)#170, avg(agg5#150)#171, avg(agg6#151)#172, avg(agg7#152)#173] -Results [11]: [i_item_id#145, ca_country#142, null AS ca_state#174, null AS county#175, avg(agg1#146)#167 AS agg1#176, avg(agg2#147)#168 AS agg2#177, avg(agg3#148)#169 AS agg3#178, avg(agg4#149)#170 AS agg4#179, avg(agg5#150)#171 AS agg5#180, avg(agg6#151)#172 AS agg6#181, avg(agg7#152)#173 AS agg7#182] - -(93) Scan parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#183, cs_bill_cdemo_sk#184, cs_item_sk#185, cs_quantity#186, cs_list_price#187, cs_sales_price#188, cs_coupon_amt#189, cs_net_profit#190, cs_sold_date_sk#191] +Input [16]: [i_item_id#116, ca_country#113, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135, sum#136, count#137] +Keys [2]: [i_item_id#116, ca_country#113] +Functions [7]: [avg(agg1#117), avg(agg2#118), avg(agg3#119), avg(agg4#120), avg(agg5#121), avg(agg6#122), avg(agg7#123)] + +(90) Scan parquet spark_catalog.default.catalog_sales +Output [9]: [cs_bill_customer_sk#138, cs_bill_cdemo_sk#139, cs_item_sk#140, cs_quantity#141, cs_list_price#142, cs_sales_price#143, cs_coupon_amt#144, cs_net_profit#145, cs_sold_date_sk#146] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#191), dynamicpruningexpression(cs_sold_date_sk#191 IN dynamicpruning#192)] +PartitionFilters: [isnotnull(cs_sold_date_sk#146), dynamicpruningexpression(cs_sold_date_sk#146 IN dynamicpruning#147)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(94) CometFilter -Input [9]: [cs_bill_customer_sk#183, cs_bill_cdemo_sk#184, cs_item_sk#185, cs_quantity#186, cs_list_price#187, cs_sales_price#188, cs_coupon_amt#189, cs_net_profit#190, cs_sold_date_sk#191] -Condition : ((isnotnull(cs_bill_cdemo_sk#184) AND isnotnull(cs_bill_customer_sk#183)) AND isnotnull(cs_item_sk#185)) +(91) CometFilter +Input [9]: [cs_bill_customer_sk#138, cs_bill_cdemo_sk#139, cs_item_sk#140, cs_quantity#141, cs_list_price#142, cs_sales_price#143, cs_coupon_amt#144, cs_net_profit#145, cs_sold_date_sk#146] +Condition : ((isnotnull(cs_bill_cdemo_sk#139) AND isnotnull(cs_bill_customer_sk#138)) AND isnotnull(cs_item_sk#140)) -(95) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#193, cd_dep_count#194] +(92) ReusedExchange [Reuses operator id: 6] +Output [2]: [cd_demo_sk#148, cd_dep_count#149] + +(93) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#138, cs_bill_cdemo_sk#139, cs_item_sk#140, cs_quantity#141, cs_list_price#142, cs_sales_price#143, cs_coupon_amt#144, cs_net_profit#145, cs_sold_date_sk#146] +Right output [2]: [cd_demo_sk#148, cd_dep_count#149] +Arguments: [cs_bill_cdemo_sk#139], [cd_demo_sk#148], Inner, BuildRight + +(94) CometProject +Input [11]: [cs_bill_customer_sk#138, cs_bill_cdemo_sk#139, cs_item_sk#140, cs_quantity#141, cs_list_price#142, cs_sales_price#143, cs_coupon_amt#144, cs_net_profit#145, cs_sold_date_sk#146, cd_demo_sk#148, cd_dep_count#149] +Arguments: [cs_bill_customer_sk#138, cs_item_sk#140, cs_quantity#141, cs_list_price#142, cs_sales_price#143, cs_coupon_amt#144, cs_net_profit#145, cs_sold_date_sk#146, cd_dep_count#149], [cs_bill_customer_sk#138, cs_item_sk#140, cs_quantity#141, cs_list_price#142, cs_sales_price#143, cs_coupon_amt#144, cs_net_profit#145, cs_sold_date_sk#146, cd_dep_count#149] + +(95) ReusedExchange [Reuses operator id: 12] +Output [4]: [c_customer_sk#150, c_current_cdemo_sk#151, c_current_addr_sk#152, c_birth_year#153] (96) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#183, cs_bill_cdemo_sk#184, cs_item_sk#185, cs_quantity#186, cs_list_price#187, cs_sales_price#188, cs_coupon_amt#189, cs_net_profit#190, cs_sold_date_sk#191] -Right output [2]: [cd_demo_sk#193, cd_dep_count#194] -Arguments: [cs_bill_cdemo_sk#184], [cd_demo_sk#193], Inner, BuildRight +Left output [9]: [cs_bill_customer_sk#138, cs_item_sk#140, cs_quantity#141, cs_list_price#142, cs_sales_price#143, cs_coupon_amt#144, cs_net_profit#145, cs_sold_date_sk#146, cd_dep_count#149] +Right output [4]: [c_customer_sk#150, c_current_cdemo_sk#151, c_current_addr_sk#152, c_birth_year#153] +Arguments: [cs_bill_customer_sk#138], [c_customer_sk#150], Inner, BuildRight (97) CometProject -Input [11]: [cs_bill_customer_sk#183, cs_bill_cdemo_sk#184, cs_item_sk#185, cs_quantity#186, cs_list_price#187, cs_sales_price#188, cs_coupon_amt#189, cs_net_profit#190, cs_sold_date_sk#191, cd_demo_sk#193, cd_dep_count#194] -Arguments: [cs_bill_customer_sk#183, cs_item_sk#185, cs_quantity#186, cs_list_price#187, cs_sales_price#188, cs_coupon_amt#189, cs_net_profit#190, cs_sold_date_sk#191, cd_dep_count#194], [cs_bill_customer_sk#183, cs_item_sk#185, cs_quantity#186, cs_list_price#187, cs_sales_price#188, cs_coupon_amt#189, cs_net_profit#190, cs_sold_date_sk#191, cd_dep_count#194] +Input [13]: [cs_bill_customer_sk#138, cs_item_sk#140, cs_quantity#141, cs_list_price#142, cs_sales_price#143, cs_coupon_amt#144, cs_net_profit#145, cs_sold_date_sk#146, cd_dep_count#149, c_customer_sk#150, c_current_cdemo_sk#151, c_current_addr_sk#152, c_birth_year#153] +Arguments: [cs_item_sk#140, cs_quantity#141, cs_list_price#142, cs_sales_price#143, cs_coupon_amt#144, cs_net_profit#145, cs_sold_date_sk#146, cd_dep_count#149, c_current_cdemo_sk#151, c_current_addr_sk#152, c_birth_year#153], [cs_item_sk#140, cs_quantity#141, cs_list_price#142, cs_sales_price#143, cs_coupon_amt#144, cs_net_profit#145, cs_sold_date_sk#146, cd_dep_count#149, c_current_cdemo_sk#151, c_current_addr_sk#152, c_birth_year#153] -(98) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#195, c_current_cdemo_sk#196, c_current_addr_sk#197, c_birth_year#198] +(98) ReusedExchange [Reuses operator id: 17] +Output [1]: [cd_demo_sk#154] (99) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#183, cs_item_sk#185, cs_quantity#186, cs_list_price#187, cs_sales_price#188, cs_coupon_amt#189, cs_net_profit#190, cs_sold_date_sk#191, cd_dep_count#194] -Right output [4]: [c_customer_sk#195, c_current_cdemo_sk#196, c_current_addr_sk#197, c_birth_year#198] -Arguments: [cs_bill_customer_sk#183], [c_customer_sk#195], Inner, BuildRight +Left output [11]: [cs_item_sk#140, cs_quantity#141, cs_list_price#142, cs_sales_price#143, cs_coupon_amt#144, cs_net_profit#145, cs_sold_date_sk#146, cd_dep_count#149, c_current_cdemo_sk#151, c_current_addr_sk#152, c_birth_year#153] +Right output [1]: [cd_demo_sk#154] +Arguments: [c_current_cdemo_sk#151], [cd_demo_sk#154], Inner, BuildRight (100) CometProject -Input [13]: [cs_bill_customer_sk#183, cs_item_sk#185, cs_quantity#186, cs_list_price#187, cs_sales_price#188, cs_coupon_amt#189, cs_net_profit#190, cs_sold_date_sk#191, cd_dep_count#194, c_customer_sk#195, c_current_cdemo_sk#196, c_current_addr_sk#197, c_birth_year#198] -Arguments: [cs_item_sk#185, cs_quantity#186, cs_list_price#187, cs_sales_price#188, cs_coupon_amt#189, cs_net_profit#190, cs_sold_date_sk#191, cd_dep_count#194, c_current_cdemo_sk#196, c_current_addr_sk#197, c_birth_year#198], [cs_item_sk#185, cs_quantity#186, cs_list_price#187, cs_sales_price#188, cs_coupon_amt#189, cs_net_profit#190, cs_sold_date_sk#191, cd_dep_count#194, c_current_cdemo_sk#196, c_current_addr_sk#197, c_birth_year#198] +Input [12]: [cs_item_sk#140, cs_quantity#141, cs_list_price#142, cs_sales_price#143, cs_coupon_amt#144, cs_net_profit#145, cs_sold_date_sk#146, cd_dep_count#149, c_current_cdemo_sk#151, c_current_addr_sk#152, c_birth_year#153, cd_demo_sk#154] +Arguments: [cs_item_sk#140, cs_quantity#141, cs_list_price#142, cs_sales_price#143, cs_coupon_amt#144, cs_net_profit#145, cs_sold_date_sk#146, cd_dep_count#149, c_current_addr_sk#152, c_birth_year#153], [cs_item_sk#140, cs_quantity#141, cs_list_price#142, cs_sales_price#143, cs_coupon_amt#144, cs_net_profit#145, cs_sold_date_sk#146, cd_dep_count#149, c_current_addr_sk#152, c_birth_year#153] -(101) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#199] - -(102) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#185, cs_quantity#186, cs_list_price#187, cs_sales_price#188, cs_coupon_amt#189, cs_net_profit#190, cs_sold_date_sk#191, cd_dep_count#194, c_current_cdemo_sk#196, c_current_addr_sk#197, c_birth_year#198] -Right output [1]: [cd_demo_sk#199] -Arguments: [c_current_cdemo_sk#196], [cd_demo_sk#199], Inner, BuildRight - -(103) CometProject -Input [12]: [cs_item_sk#185, cs_quantity#186, cs_list_price#187, cs_sales_price#188, cs_coupon_amt#189, cs_net_profit#190, cs_sold_date_sk#191, cd_dep_count#194, c_current_cdemo_sk#196, c_current_addr_sk#197, c_birth_year#198, cd_demo_sk#199] -Arguments: [cs_item_sk#185, cs_quantity#186, cs_list_price#187, cs_sales_price#188, cs_coupon_amt#189, cs_net_profit#190, cs_sold_date_sk#191, cd_dep_count#194, c_current_addr_sk#197, c_birth_year#198], [cs_item_sk#185, cs_quantity#186, cs_list_price#187, cs_sales_price#188, cs_coupon_amt#189, cs_net_profit#190, cs_sold_date_sk#191, cd_dep_count#194, c_current_addr_sk#197, c_birth_year#198] - -(104) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#200, ca_state#201] +(101) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#155, ca_state#156] 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 -(105) CometFilter -Input [2]: [ca_address_sk#200, ca_state#201] -Condition : (ca_state#201 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#200)) +(102) CometFilter +Input [2]: [ca_address_sk#155, ca_state#156] +Condition : (ca_state#156 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#155)) + +(103) CometProject +Input [2]: [ca_address_sk#155, ca_state#156] +Arguments: [ca_address_sk#155], [ca_address_sk#155] + +(104) CometBroadcastExchange +Input [1]: [ca_address_sk#155] +Arguments: [ca_address_sk#155] + +(105) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#140, cs_quantity#141, cs_list_price#142, cs_sales_price#143, cs_coupon_amt#144, cs_net_profit#145, cs_sold_date_sk#146, cd_dep_count#149, c_current_addr_sk#152, c_birth_year#153] +Right output [1]: [ca_address_sk#155] +Arguments: [c_current_addr_sk#152], [ca_address_sk#155], Inner, BuildRight (106) CometProject -Input [2]: [ca_address_sk#200, ca_state#201] -Arguments: [ca_address_sk#200], [ca_address_sk#200] +Input [11]: [cs_item_sk#140, cs_quantity#141, cs_list_price#142, cs_sales_price#143, cs_coupon_amt#144, cs_net_profit#145, cs_sold_date_sk#146, cd_dep_count#149, c_current_addr_sk#152, c_birth_year#153, ca_address_sk#155] +Arguments: [cs_item_sk#140, cs_quantity#141, cs_list_price#142, cs_sales_price#143, cs_coupon_amt#144, cs_net_profit#145, cs_sold_date_sk#146, cd_dep_count#149, c_birth_year#153], [cs_item_sk#140, cs_quantity#141, cs_list_price#142, cs_sales_price#143, cs_coupon_amt#144, cs_net_profit#145, cs_sold_date_sk#146, cd_dep_count#149, c_birth_year#153] -(107) CometBroadcastExchange -Input [1]: [ca_address_sk#200] -Arguments: [ca_address_sk#200] +(107) ReusedExchange [Reuses operator id: 28] +Output [1]: [d_date_sk#157] (108) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#185, cs_quantity#186, cs_list_price#187, cs_sales_price#188, cs_coupon_amt#189, cs_net_profit#190, cs_sold_date_sk#191, cd_dep_count#194, c_current_addr_sk#197, c_birth_year#198] -Right output [1]: [ca_address_sk#200] -Arguments: [c_current_addr_sk#197], [ca_address_sk#200], Inner, BuildRight +Left output [9]: [cs_item_sk#140, cs_quantity#141, cs_list_price#142, cs_sales_price#143, cs_coupon_amt#144, cs_net_profit#145, cs_sold_date_sk#146, cd_dep_count#149, c_birth_year#153] +Right output [1]: [d_date_sk#157] +Arguments: [cs_sold_date_sk#146], [d_date_sk#157], Inner, BuildRight (109) CometProject -Input [11]: [cs_item_sk#185, cs_quantity#186, cs_list_price#187, cs_sales_price#188, cs_coupon_amt#189, cs_net_profit#190, cs_sold_date_sk#191, cd_dep_count#194, c_current_addr_sk#197, c_birth_year#198, ca_address_sk#200] -Arguments: [cs_item_sk#185, cs_quantity#186, cs_list_price#187, cs_sales_price#188, cs_coupon_amt#189, cs_net_profit#190, cs_sold_date_sk#191, cd_dep_count#194, c_birth_year#198], [cs_item_sk#185, cs_quantity#186, cs_list_price#187, cs_sales_price#188, cs_coupon_amt#189, cs_net_profit#190, cs_sold_date_sk#191, cd_dep_count#194, c_birth_year#198] +Input [10]: [cs_item_sk#140, cs_quantity#141, cs_list_price#142, cs_sales_price#143, cs_coupon_amt#144, cs_net_profit#145, cs_sold_date_sk#146, cd_dep_count#149, c_birth_year#153, d_date_sk#157] +Arguments: [cs_item_sk#140, cs_quantity#141, cs_list_price#142, cs_sales_price#143, cs_coupon_amt#144, cs_net_profit#145, cd_dep_count#149, c_birth_year#153], [cs_item_sk#140, cs_quantity#141, cs_list_price#142, cs_sales_price#143, cs_coupon_amt#144, cs_net_profit#145, cd_dep_count#149, c_birth_year#153] -(110) ReusedExchange [Reuses operator id: 28] -Output [1]: [d_date_sk#202] +(110) ReusedExchange [Reuses operator id: 33] +Output [2]: [i_item_sk#158, i_item_id#159] (111) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#185, cs_quantity#186, cs_list_price#187, cs_sales_price#188, cs_coupon_amt#189, cs_net_profit#190, cs_sold_date_sk#191, cd_dep_count#194, c_birth_year#198] -Right output [1]: [d_date_sk#202] -Arguments: [cs_sold_date_sk#191], [d_date_sk#202], Inner, BuildRight +Left output [8]: [cs_item_sk#140, cs_quantity#141, cs_list_price#142, cs_sales_price#143, cs_coupon_amt#144, cs_net_profit#145, cd_dep_count#149, c_birth_year#153] +Right output [2]: [i_item_sk#158, i_item_id#159] +Arguments: [cs_item_sk#140], [i_item_sk#158], Inner, BuildRight (112) CometProject -Input [10]: [cs_item_sk#185, cs_quantity#186, cs_list_price#187, cs_sales_price#188, cs_coupon_amt#189, cs_net_profit#190, cs_sold_date_sk#191, cd_dep_count#194, c_birth_year#198, d_date_sk#202] -Arguments: [cs_item_sk#185, cs_quantity#186, cs_list_price#187, cs_sales_price#188, cs_coupon_amt#189, cs_net_profit#190, cd_dep_count#194, c_birth_year#198], [cs_item_sk#185, cs_quantity#186, cs_list_price#187, cs_sales_price#188, cs_coupon_amt#189, cs_net_profit#190, cd_dep_count#194, c_birth_year#198] - -(113) ReusedExchange [Reuses operator id: 33] -Output [2]: [i_item_sk#203, i_item_id#204] - -(114) CometBroadcastHashJoin -Left output [8]: [cs_item_sk#185, cs_quantity#186, cs_list_price#187, cs_sales_price#188, cs_coupon_amt#189, cs_net_profit#190, cd_dep_count#194, c_birth_year#198] -Right output [2]: [i_item_sk#203, i_item_id#204] -Arguments: [cs_item_sk#185], [i_item_sk#203], Inner, BuildRight - -(115) CometProject -Input [10]: [cs_item_sk#185, cs_quantity#186, cs_list_price#187, cs_sales_price#188, cs_coupon_amt#189, cs_net_profit#190, cd_dep_count#194, c_birth_year#198, i_item_sk#203, i_item_id#204] -Arguments: [i_item_id#204, agg1#205, agg2#206, agg3#207, agg4#208, agg5#209, agg6#210, agg7#211], [i_item_id#204, cast(cs_quantity#186 as decimal(12,2)) AS agg1#205, cast(cs_list_price#187 as decimal(12,2)) AS agg2#206, cast(cs_coupon_amt#189 as decimal(12,2)) AS agg3#207, cast(cs_sales_price#188 as decimal(12,2)) AS agg4#208, cast(cs_net_profit#190 as decimal(12,2)) AS agg5#209, cast(c_birth_year#198 as decimal(12,2)) AS agg6#210, cast(cd_dep_count#194 as decimal(12,2)) AS agg7#211] - -(116) CometHashAggregate -Input [8]: [i_item_id#204, agg1#205, agg2#206, agg3#207, agg4#208, agg5#209, agg6#210, agg7#211] -Keys [1]: [i_item_id#204] -Functions [7]: [partial_avg(agg1#205), partial_avg(agg2#206), partial_avg(agg3#207), partial_avg(agg4#208), partial_avg(agg5#209), partial_avg(agg6#210), partial_avg(agg7#211)] - -(117) ColumnarToRow [codegen id : 7] -Input [15]: [i_item_id#204, sum#212, count#213, sum#214, count#215, sum#216, count#217, sum#218, count#219, sum#220, count#221, sum#222, count#223, sum#224, count#225] - -(118) Exchange -Input [15]: [i_item_id#204, sum#212, count#213, sum#214, count#215, sum#216, count#217, sum#218, count#219, sum#220, count#221, sum#222, count#223, sum#224, count#225] -Arguments: hashpartitioning(i_item_id#204, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(119) HashAggregate [codegen id : 8] -Input [15]: [i_item_id#204, sum#212, count#213, sum#214, count#215, sum#216, count#217, sum#218, count#219, sum#220, count#221, sum#222, count#223, sum#224, count#225] -Keys [1]: [i_item_id#204] -Functions [7]: [avg(agg1#205), avg(agg2#206), avg(agg3#207), avg(agg4#208), avg(agg5#209), avg(agg6#210), avg(agg7#211)] -Aggregate Attributes [7]: [avg(agg1#205)#226, avg(agg2#206)#227, avg(agg3#207)#228, avg(agg4#208)#229, avg(agg5#209)#230, avg(agg6#210)#231, avg(agg7#211)#232] -Results [11]: [i_item_id#204, null AS ca_country#233, null AS ca_state#234, null AS county#235, avg(agg1#205)#226 AS agg1#236, avg(agg2#206)#227 AS agg2#237, avg(agg3#207)#228 AS agg3#238, avg(agg4#208)#229 AS agg4#239, avg(agg5#209)#230 AS agg5#240, avg(agg6#210)#231 AS agg6#241, avg(agg7#211)#232 AS agg7#242] - -(120) Scan parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#243, cs_bill_cdemo_sk#244, cs_item_sk#245, cs_quantity#246, cs_list_price#247, cs_sales_price#248, cs_coupon_amt#249, cs_net_profit#250, cs_sold_date_sk#251] +Input [10]: [cs_item_sk#140, cs_quantity#141, cs_list_price#142, cs_sales_price#143, cs_coupon_amt#144, cs_net_profit#145, cd_dep_count#149, c_birth_year#153, i_item_sk#158, i_item_id#159] +Arguments: [i_item_id#159, agg1#160, agg2#161, agg3#162, agg4#163, agg5#164, agg6#165, agg7#166], [i_item_id#159, cast(cs_quantity#141 as decimal(12,2)) AS agg1#160, cast(cs_list_price#142 as decimal(12,2)) AS agg2#161, cast(cs_coupon_amt#144 as decimal(12,2)) AS agg3#162, cast(cs_sales_price#143 as decimal(12,2)) AS agg4#163, cast(cs_net_profit#145 as decimal(12,2)) AS agg5#164, cast(c_birth_year#153 as decimal(12,2)) AS agg6#165, cast(cd_dep_count#149 as decimal(12,2)) AS agg7#166] + +(113) CometHashAggregate +Input [8]: [i_item_id#159, agg1#160, agg2#161, agg3#162, agg4#163, agg5#164, agg6#165, agg7#166] +Keys [1]: [i_item_id#159] +Functions [7]: [partial_avg(agg1#160), partial_avg(agg2#161), partial_avg(agg3#162), partial_avg(agg4#163), partial_avg(agg5#164), partial_avg(agg6#165), partial_avg(agg7#166)] + +(114) CometColumnarExchange +Input [15]: [i_item_id#159, 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] +Arguments: hashpartitioning(i_item_id#159, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(115) CometHashAggregate +Input [15]: [i_item_id#159, 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] +Keys [1]: [i_item_id#159] +Functions [7]: [avg(agg1#160), avg(agg2#161), avg(agg3#162), avg(agg4#163), avg(agg5#164), avg(agg6#165), avg(agg7#166)] + +(116) Scan parquet spark_catalog.default.catalog_sales +Output [9]: [cs_bill_customer_sk#181, cs_bill_cdemo_sk#182, cs_item_sk#183, cs_quantity#184, cs_list_price#185, cs_sales_price#186, cs_coupon_amt#187, cs_net_profit#188, cs_sold_date_sk#189] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#251), dynamicpruningexpression(cs_sold_date_sk#251 IN dynamicpruning#252)] +PartitionFilters: [isnotnull(cs_sold_date_sk#189), dynamicpruningexpression(cs_sold_date_sk#189 IN dynamicpruning#190)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(121) CometFilter -Input [9]: [cs_bill_customer_sk#243, cs_bill_cdemo_sk#244, cs_item_sk#245, cs_quantity#246, cs_list_price#247, cs_sales_price#248, cs_coupon_amt#249, cs_net_profit#250, cs_sold_date_sk#251] -Condition : ((isnotnull(cs_bill_cdemo_sk#244) AND isnotnull(cs_bill_customer_sk#243)) AND isnotnull(cs_item_sk#245)) +(117) CometFilter +Input [9]: [cs_bill_customer_sk#181, cs_bill_cdemo_sk#182, cs_item_sk#183, cs_quantity#184, cs_list_price#185, cs_sales_price#186, cs_coupon_amt#187, cs_net_profit#188, cs_sold_date_sk#189] +Condition : ((isnotnull(cs_bill_cdemo_sk#182) AND isnotnull(cs_bill_customer_sk#181)) AND isnotnull(cs_item_sk#183)) -(122) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#253, cd_dep_count#254] +(118) ReusedExchange [Reuses operator id: 6] +Output [2]: [cd_demo_sk#191, cd_dep_count#192] -(123) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#243, cs_bill_cdemo_sk#244, cs_item_sk#245, cs_quantity#246, cs_list_price#247, cs_sales_price#248, cs_coupon_amt#249, cs_net_profit#250, cs_sold_date_sk#251] -Right output [2]: [cd_demo_sk#253, cd_dep_count#254] -Arguments: [cs_bill_cdemo_sk#244], [cd_demo_sk#253], Inner, BuildRight +(119) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#181, cs_bill_cdemo_sk#182, cs_item_sk#183, cs_quantity#184, cs_list_price#185, cs_sales_price#186, cs_coupon_amt#187, cs_net_profit#188, cs_sold_date_sk#189] +Right output [2]: [cd_demo_sk#191, cd_dep_count#192] +Arguments: [cs_bill_cdemo_sk#182], [cd_demo_sk#191], Inner, BuildRight -(124) CometProject -Input [11]: [cs_bill_customer_sk#243, cs_bill_cdemo_sk#244, cs_item_sk#245, cs_quantity#246, cs_list_price#247, cs_sales_price#248, cs_coupon_amt#249, cs_net_profit#250, cs_sold_date_sk#251, cd_demo_sk#253, cd_dep_count#254] -Arguments: [cs_bill_customer_sk#243, cs_item_sk#245, cs_quantity#246, cs_list_price#247, cs_sales_price#248, cs_coupon_amt#249, cs_net_profit#250, cs_sold_date_sk#251, cd_dep_count#254], [cs_bill_customer_sk#243, cs_item_sk#245, cs_quantity#246, cs_list_price#247, cs_sales_price#248, cs_coupon_amt#249, cs_net_profit#250, cs_sold_date_sk#251, cd_dep_count#254] +(120) CometProject +Input [11]: [cs_bill_customer_sk#181, cs_bill_cdemo_sk#182, cs_item_sk#183, cs_quantity#184, cs_list_price#185, cs_sales_price#186, cs_coupon_amt#187, cs_net_profit#188, cs_sold_date_sk#189, cd_demo_sk#191, cd_dep_count#192] +Arguments: [cs_bill_customer_sk#181, cs_item_sk#183, cs_quantity#184, cs_list_price#185, cs_sales_price#186, cs_coupon_amt#187, cs_net_profit#188, cs_sold_date_sk#189, cd_dep_count#192], [cs_bill_customer_sk#181, cs_item_sk#183, cs_quantity#184, cs_list_price#185, cs_sales_price#186, cs_coupon_amt#187, cs_net_profit#188, cs_sold_date_sk#189, cd_dep_count#192] -(125) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#255, c_current_cdemo_sk#256, c_current_addr_sk#257, c_birth_year#258] +(121) ReusedExchange [Reuses operator id: 12] +Output [4]: [c_customer_sk#193, c_current_cdemo_sk#194, c_current_addr_sk#195, c_birth_year#196] -(126) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#243, cs_item_sk#245, cs_quantity#246, cs_list_price#247, cs_sales_price#248, cs_coupon_amt#249, cs_net_profit#250, cs_sold_date_sk#251, cd_dep_count#254] -Right output [4]: [c_customer_sk#255, c_current_cdemo_sk#256, c_current_addr_sk#257, c_birth_year#258] -Arguments: [cs_bill_customer_sk#243], [c_customer_sk#255], Inner, BuildRight +(122) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#181, cs_item_sk#183, cs_quantity#184, cs_list_price#185, cs_sales_price#186, cs_coupon_amt#187, cs_net_profit#188, cs_sold_date_sk#189, cd_dep_count#192] +Right output [4]: [c_customer_sk#193, c_current_cdemo_sk#194, c_current_addr_sk#195, c_birth_year#196] +Arguments: [cs_bill_customer_sk#181], [c_customer_sk#193], Inner, BuildRight -(127) CometProject -Input [13]: [cs_bill_customer_sk#243, cs_item_sk#245, cs_quantity#246, cs_list_price#247, cs_sales_price#248, cs_coupon_amt#249, cs_net_profit#250, cs_sold_date_sk#251, cd_dep_count#254, c_customer_sk#255, c_current_cdemo_sk#256, c_current_addr_sk#257, c_birth_year#258] -Arguments: [cs_item_sk#245, cs_quantity#246, cs_list_price#247, cs_sales_price#248, cs_coupon_amt#249, cs_net_profit#250, cs_sold_date_sk#251, cd_dep_count#254, c_current_cdemo_sk#256, c_current_addr_sk#257, c_birth_year#258], [cs_item_sk#245, cs_quantity#246, cs_list_price#247, cs_sales_price#248, cs_coupon_amt#249, cs_net_profit#250, cs_sold_date_sk#251, cd_dep_count#254, c_current_cdemo_sk#256, c_current_addr_sk#257, c_birth_year#258] +(123) CometProject +Input [13]: [cs_bill_customer_sk#181, cs_item_sk#183, cs_quantity#184, cs_list_price#185, cs_sales_price#186, cs_coupon_amt#187, cs_net_profit#188, cs_sold_date_sk#189, cd_dep_count#192, c_customer_sk#193, c_current_cdemo_sk#194, c_current_addr_sk#195, c_birth_year#196] +Arguments: [cs_item_sk#183, cs_quantity#184, cs_list_price#185, cs_sales_price#186, cs_coupon_amt#187, cs_net_profit#188, cs_sold_date_sk#189, cd_dep_count#192, c_current_cdemo_sk#194, c_current_addr_sk#195, c_birth_year#196], [cs_item_sk#183, cs_quantity#184, cs_list_price#185, cs_sales_price#186, cs_coupon_amt#187, cs_net_profit#188, cs_sold_date_sk#189, cd_dep_count#192, c_current_cdemo_sk#194, c_current_addr_sk#195, c_birth_year#196] -(128) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#259] +(124) ReusedExchange [Reuses operator id: 17] +Output [1]: [cd_demo_sk#197] -(129) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#245, cs_quantity#246, cs_list_price#247, cs_sales_price#248, cs_coupon_amt#249, cs_net_profit#250, cs_sold_date_sk#251, cd_dep_count#254, c_current_cdemo_sk#256, c_current_addr_sk#257, c_birth_year#258] -Right output [1]: [cd_demo_sk#259] -Arguments: [c_current_cdemo_sk#256], [cd_demo_sk#259], Inner, BuildRight +(125) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#183, cs_quantity#184, cs_list_price#185, cs_sales_price#186, cs_coupon_amt#187, cs_net_profit#188, cs_sold_date_sk#189, cd_dep_count#192, c_current_cdemo_sk#194, c_current_addr_sk#195, c_birth_year#196] +Right output [1]: [cd_demo_sk#197] +Arguments: [c_current_cdemo_sk#194], [cd_demo_sk#197], Inner, BuildRight -(130) CometProject -Input [12]: [cs_item_sk#245, cs_quantity#246, cs_list_price#247, cs_sales_price#248, cs_coupon_amt#249, cs_net_profit#250, cs_sold_date_sk#251, cd_dep_count#254, c_current_cdemo_sk#256, c_current_addr_sk#257, c_birth_year#258, cd_demo_sk#259] -Arguments: [cs_item_sk#245, cs_quantity#246, cs_list_price#247, cs_sales_price#248, cs_coupon_amt#249, cs_net_profit#250, cs_sold_date_sk#251, cd_dep_count#254, c_current_addr_sk#257, c_birth_year#258], [cs_item_sk#245, cs_quantity#246, cs_list_price#247, cs_sales_price#248, cs_coupon_amt#249, cs_net_profit#250, cs_sold_date_sk#251, cd_dep_count#254, c_current_addr_sk#257, c_birth_year#258] +(126) CometProject +Input [12]: [cs_item_sk#183, cs_quantity#184, cs_list_price#185, cs_sales_price#186, cs_coupon_amt#187, cs_net_profit#188, cs_sold_date_sk#189, cd_dep_count#192, c_current_cdemo_sk#194, c_current_addr_sk#195, c_birth_year#196, cd_demo_sk#197] +Arguments: [cs_item_sk#183, cs_quantity#184, cs_list_price#185, cs_sales_price#186, cs_coupon_amt#187, cs_net_profit#188, cs_sold_date_sk#189, cd_dep_count#192, c_current_addr_sk#195, c_birth_year#196], [cs_item_sk#183, cs_quantity#184, cs_list_price#185, cs_sales_price#186, cs_coupon_amt#187, cs_net_profit#188, cs_sold_date_sk#189, cd_dep_count#192, c_current_addr_sk#195, c_birth_year#196] -(131) ReusedExchange [Reuses operator id: 107] -Output [1]: [ca_address_sk#260] +(127) ReusedExchange [Reuses operator id: 104] +Output [1]: [ca_address_sk#198] -(132) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#245, cs_quantity#246, cs_list_price#247, cs_sales_price#248, cs_coupon_amt#249, cs_net_profit#250, cs_sold_date_sk#251, cd_dep_count#254, c_current_addr_sk#257, c_birth_year#258] -Right output [1]: [ca_address_sk#260] -Arguments: [c_current_addr_sk#257], [ca_address_sk#260], Inner, BuildRight +(128) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#183, cs_quantity#184, cs_list_price#185, cs_sales_price#186, cs_coupon_amt#187, cs_net_profit#188, cs_sold_date_sk#189, cd_dep_count#192, c_current_addr_sk#195, c_birth_year#196] +Right output [1]: [ca_address_sk#198] +Arguments: [c_current_addr_sk#195], [ca_address_sk#198], Inner, BuildRight -(133) CometProject -Input [11]: [cs_item_sk#245, cs_quantity#246, cs_list_price#247, cs_sales_price#248, cs_coupon_amt#249, cs_net_profit#250, cs_sold_date_sk#251, cd_dep_count#254, c_current_addr_sk#257, c_birth_year#258, ca_address_sk#260] -Arguments: [cs_item_sk#245, cs_quantity#246, cs_list_price#247, cs_sales_price#248, cs_coupon_amt#249, cs_net_profit#250, cs_sold_date_sk#251, cd_dep_count#254, c_birth_year#258], [cs_item_sk#245, cs_quantity#246, cs_list_price#247, cs_sales_price#248, cs_coupon_amt#249, cs_net_profit#250, cs_sold_date_sk#251, cd_dep_count#254, c_birth_year#258] +(129) CometProject +Input [11]: [cs_item_sk#183, cs_quantity#184, cs_list_price#185, cs_sales_price#186, cs_coupon_amt#187, cs_net_profit#188, cs_sold_date_sk#189, cd_dep_count#192, c_current_addr_sk#195, c_birth_year#196, ca_address_sk#198] +Arguments: [cs_item_sk#183, cs_quantity#184, cs_list_price#185, cs_sales_price#186, cs_coupon_amt#187, cs_net_profit#188, cs_sold_date_sk#189, cd_dep_count#192, c_birth_year#196], [cs_item_sk#183, cs_quantity#184, cs_list_price#185, cs_sales_price#186, cs_coupon_amt#187, cs_net_profit#188, cs_sold_date_sk#189, cd_dep_count#192, c_birth_year#196] -(134) ReusedExchange [Reuses operator id: 28] -Output [1]: [d_date_sk#261] +(130) ReusedExchange [Reuses operator id: 28] +Output [1]: [d_date_sk#199] -(135) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#245, cs_quantity#246, cs_list_price#247, cs_sales_price#248, cs_coupon_amt#249, cs_net_profit#250, cs_sold_date_sk#251, cd_dep_count#254, c_birth_year#258] -Right output [1]: [d_date_sk#261] -Arguments: [cs_sold_date_sk#251], [d_date_sk#261], Inner, BuildRight +(131) CometBroadcastHashJoin +Left output [9]: [cs_item_sk#183, cs_quantity#184, cs_list_price#185, cs_sales_price#186, cs_coupon_amt#187, cs_net_profit#188, cs_sold_date_sk#189, cd_dep_count#192, c_birth_year#196] +Right output [1]: [d_date_sk#199] +Arguments: [cs_sold_date_sk#189], [d_date_sk#199], Inner, BuildRight -(136) CometProject -Input [10]: [cs_item_sk#245, cs_quantity#246, cs_list_price#247, cs_sales_price#248, cs_coupon_amt#249, cs_net_profit#250, cs_sold_date_sk#251, cd_dep_count#254, c_birth_year#258, d_date_sk#261] -Arguments: [cs_item_sk#245, cs_quantity#246, cs_list_price#247, cs_sales_price#248, cs_coupon_amt#249, cs_net_profit#250, cd_dep_count#254, c_birth_year#258], [cs_item_sk#245, cs_quantity#246, cs_list_price#247, cs_sales_price#248, cs_coupon_amt#249, cs_net_profit#250, cd_dep_count#254, c_birth_year#258] +(132) CometProject +Input [10]: [cs_item_sk#183, cs_quantity#184, cs_list_price#185, cs_sales_price#186, cs_coupon_amt#187, cs_net_profit#188, cs_sold_date_sk#189, cd_dep_count#192, c_birth_year#196, d_date_sk#199] +Arguments: [cs_item_sk#183, cs_quantity#184, cs_list_price#185, cs_sales_price#186, cs_coupon_amt#187, cs_net_profit#188, cd_dep_count#192, c_birth_year#196], [cs_item_sk#183, cs_quantity#184, cs_list_price#185, cs_sales_price#186, cs_coupon_amt#187, cs_net_profit#188, cd_dep_count#192, c_birth_year#196] -(137) Scan parquet spark_catalog.default.item -Output [1]: [i_item_sk#262] +(133) Scan parquet spark_catalog.default.item +Output [1]: [i_item_sk#200] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(138) CometFilter -Input [1]: [i_item_sk#262] -Condition : isnotnull(i_item_sk#262) +(134) CometFilter +Input [1]: [i_item_sk#200] +Condition : isnotnull(i_item_sk#200) -(139) CometBroadcastExchange -Input [1]: [i_item_sk#262] -Arguments: [i_item_sk#262] +(135) CometBroadcastExchange +Input [1]: [i_item_sk#200] +Arguments: [i_item_sk#200] -(140) CometBroadcastHashJoin -Left output [8]: [cs_item_sk#245, cs_quantity#246, cs_list_price#247, cs_sales_price#248, cs_coupon_amt#249, cs_net_profit#250, cd_dep_count#254, c_birth_year#258] -Right output [1]: [i_item_sk#262] -Arguments: [cs_item_sk#245], [i_item_sk#262], Inner, BuildRight +(136) CometBroadcastHashJoin +Left output [8]: [cs_item_sk#183, cs_quantity#184, cs_list_price#185, cs_sales_price#186, cs_coupon_amt#187, cs_net_profit#188, cd_dep_count#192, c_birth_year#196] +Right output [1]: [i_item_sk#200] +Arguments: [cs_item_sk#183], [i_item_sk#200], Inner, BuildRight -(141) CometProject -Input [9]: [cs_item_sk#245, cs_quantity#246, cs_list_price#247, cs_sales_price#248, cs_coupon_amt#249, cs_net_profit#250, cd_dep_count#254, c_birth_year#258, i_item_sk#262] -Arguments: [agg1#263, agg2#264, agg3#265, agg4#266, agg5#267, agg6#268, agg7#269], [cast(cs_quantity#246 as decimal(12,2)) AS agg1#263, cast(cs_list_price#247 as decimal(12,2)) AS agg2#264, cast(cs_coupon_amt#249 as decimal(12,2)) AS agg3#265, cast(cs_sales_price#248 as decimal(12,2)) AS agg4#266, cast(cs_net_profit#250 as decimal(12,2)) AS agg5#267, cast(c_birth_year#258 as decimal(12,2)) AS agg6#268, cast(cd_dep_count#254 as decimal(12,2)) AS agg7#269] +(137) CometProject +Input [9]: [cs_item_sk#183, cs_quantity#184, cs_list_price#185, cs_sales_price#186, cs_coupon_amt#187, cs_net_profit#188, cd_dep_count#192, c_birth_year#196, i_item_sk#200] +Arguments: [agg1#201, agg2#202, agg3#203, agg4#204, agg5#205, agg6#206, agg7#207], [cast(cs_quantity#184 as decimal(12,2)) AS agg1#201, cast(cs_list_price#185 as decimal(12,2)) AS agg2#202, cast(cs_coupon_amt#187 as decimal(12,2)) AS agg3#203, cast(cs_sales_price#186 as decimal(12,2)) AS agg4#204, cast(cs_net_profit#188 as decimal(12,2)) AS agg5#205, cast(c_birth_year#196 as decimal(12,2)) AS agg6#206, cast(cd_dep_count#192 as decimal(12,2)) AS agg7#207] -(142) CometHashAggregate -Input [7]: [agg1#263, agg2#264, agg3#265, agg4#266, agg5#267, agg6#268, agg7#269] +(138) CometHashAggregate +Input [7]: [agg1#201, agg2#202, agg3#203, agg4#204, agg5#205, agg6#206, agg7#207] Keys: [] -Functions [7]: [partial_avg(agg1#263), partial_avg(agg2#264), partial_avg(agg3#265), partial_avg(agg4#266), partial_avg(agg5#267), partial_avg(agg6#268), partial_avg(agg7#269)] +Functions [7]: [partial_avg(agg1#201), partial_avg(agg2#202), partial_avg(agg3#203), partial_avg(agg4#204), partial_avg(agg5#205), partial_avg(agg6#206), partial_avg(agg7#207)] -(143) ColumnarToRow [codegen id : 9] -Input [14]: [sum#270, count#271, sum#272, count#273, sum#274, count#275, sum#276, count#277, sum#278, count#279, sum#280, count#281, sum#282, count#283] +(139) CometColumnarExchange +Input [14]: [sum#208, count#209, sum#210, count#211, sum#212, count#213, sum#214, count#215, sum#216, count#217, sum#218, count#219, sum#220, count#221] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(144) Exchange -Input [14]: [sum#270, count#271, sum#272, count#273, sum#274, count#275, sum#276, count#277, sum#278, count#279, sum#280, count#281, sum#282, count#283] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5] - -(145) HashAggregate [codegen id : 10] -Input [14]: [sum#270, count#271, sum#272, count#273, sum#274, count#275, sum#276, count#277, sum#278, count#279, sum#280, count#281, sum#282, count#283] +(140) CometHashAggregate +Input [14]: [sum#208, count#209, sum#210, count#211, sum#212, count#213, sum#214, count#215, sum#216, count#217, sum#218, count#219, sum#220, count#221] Keys: [] -Functions [7]: [avg(agg1#263), avg(agg2#264), avg(agg3#265), avg(agg4#266), avg(agg5#267), avg(agg6#268), avg(agg7#269)] -Aggregate Attributes [7]: [avg(agg1#263)#284, avg(agg2#264)#285, avg(agg3#265)#286, avg(agg4#266)#287, avg(agg5#267)#288, avg(agg6#268)#289, avg(agg7#269)#290] -Results [11]: [null AS i_item_id#291, null AS ca_country#292, null AS ca_state#293, null AS county#294, avg(agg1#263)#284 AS agg1#295, avg(agg2#264)#285 AS agg2#296, avg(agg3#265)#286 AS agg3#297, avg(agg4#266)#287 AS agg4#298, avg(agg5#267)#288 AS agg5#299, avg(agg6#268)#289 AS agg6#300, avg(agg7#269)#290 AS agg7#301] +Functions [7]: [avg(agg1#201), avg(agg2#202), avg(agg3#203), avg(agg4#204), avg(agg5#205), avg(agg6#206), avg(agg7#207)] + +(141) CometUnion +Child 0 Input [11]: [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, agg1#222, agg2#223, agg3#224, agg4#225, agg5#226, agg6#227, agg7#228] +Child 1 Input [11]: [i_item_id#72, ca_country#69, ca_state#68, county#229, agg1#230, agg2#231, agg3#232, agg4#233, agg5#234, agg6#235, agg7#236] +Child 2 Input [11]: [i_item_id#116, ca_country#113, ca_state#237, county#238, agg1#239, agg2#240, agg3#241, agg4#242, agg5#243, agg6#244, agg7#245] +Child 3 Input [11]: [i_item_id#159, ca_country#246, ca_state#247, county#248, agg1#249, agg2#250, agg3#251, agg4#252, agg5#253, agg6#254, agg7#255] +Child 4 Input [11]: [i_item_id#256, ca_country#257, ca_state#258, county#259, agg1#260, agg2#261, agg3#262, agg4#263, agg5#264, agg6#265, agg7#266] -(146) Union +(142) CometTakeOrderedAndProject +Input [11]: [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, agg1#222, agg2#223, agg3#224, agg4#225, agg5#226, agg6#227, agg7#228] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[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], output=[i_item_id#28,ca_country#24,ca_state#23,ca_county#22,agg1#222,agg2#223,agg3#224,agg4#225,agg5#226,agg6#227,agg7#228]), [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, agg1#222, agg2#223, agg3#224, agg4#225, agg5#226, agg6#227, agg7#228], 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#222, agg2#223, agg3#224, agg4#225, agg5#226, agg6#227, agg7#228] -(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] +(143) ColumnarToRow [codegen id : 1] +Input [11]: [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, agg1#222, agg2#223, agg3#224, agg4#225, agg5#226, agg6#227, agg7#228] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (152) -+- * ColumnarToRow (151) - +- CometProject (150) - +- CometFilter (149) - +- CometScan parquet spark_catalog.default.date_dim (148) +BroadcastExchange (148) ++- * ColumnarToRow (147) + +- CometProject (146) + +- CometFilter (145) + +- CometScan parquet spark_catalog.default.date_dim (144) -(148) Scan parquet spark_catalog.default.date_dim +(144) 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 -(149) CometFilter +(145) 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)) -(150) CometProject +(146) CometProject Input [2]: [d_date_sk#25, d_year#26] Arguments: [d_date_sk#25], [d_date_sk#25] -(151) ColumnarToRow [codegen id : 1] +(147) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] -(152) BroadcastExchange +(148) BroadcastExchange Input [1]: [d_date_sk#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#72 IN dynamicpruning#10 +Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#58 IN dynamicpruning#10 -Subquery:3 Hosting operator id = 66 Hosting Expression = cs_sold_date_sk#131 IN dynamicpruning#10 +Subquery:3 Hosting operator id = 64 Hosting Expression = cs_sold_date_sk#102 IN dynamicpruning#10 -Subquery:4 Hosting operator id = 93 Hosting Expression = cs_sold_date_sk#191 IN dynamicpruning#10 +Subquery:4 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#146 IN dynamicpruning#10 -Subquery:5 Hosting operator id = 120 Hosting Expression = cs_sold_date_sk#251 IN dynamicpruning#10 +Subquery:5 Hosting operator id = 116 Hosting Expression = cs_sold_date_sk#189 IN dynamicpruning#10 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/simplified.txt index f854b2b31..aef60ba9b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/simplified.txt @@ -1,179 +1,157 @@ -TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - Union - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,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,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,i_item_sk,i_item_id] - 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_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,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 [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,ca_address_sk,ca_county,ca_state,ca_country] - 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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - 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_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,cd_demo_sk,cd_dep_count] - CometFilter [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] - 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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange [cd_demo_sk] #5 - CometFilter [cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #8 - CometFilter [i_item_sk,i_item_id] - 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] #9 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,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,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country,i_item_sk,i_item_id] - 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_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,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 [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,ca_address_sk,ca_state,ca_country] - 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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - 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_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,cd_demo_sk,cd_dep_count] - CometFilter [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] - 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 - 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 [ca_address_sk,ca_state,ca_country] #10 - CometFilter [ca_address_sk,ca_state,ca_country] - 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] #11 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,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,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country,i_item_sk,i_item_id] - 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_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,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 [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,ca_address_sk,ca_country] - 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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - 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_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,cd_demo_sk,cd_dep_count] - CometFilter [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] - 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 - 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 [ca_address_sk,ca_country] #12 - CometProject [ca_address_sk,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] - 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] #13 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,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,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk,i_item_id] - 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_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,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 [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,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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - 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_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,cd_demo_sk,cd_dep_count] - CometFilter [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] - 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 - 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 [ca_address_sk] #14 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - 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 #15 - WholeStageCodegen (9) - ColumnarToRow - InputAdapter - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,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,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,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_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,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 [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,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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - 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_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,cd_demo_sk,cd_dep_count] - CometFilter [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] - 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 - 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 [i_item_sk] #16 - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometUnion [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [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,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] + CometColumnarExchange [i_item_id,ca_country,ca_state,ca_county] #1 + CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,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,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,i_item_sk,i_item_id] + 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_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,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 [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,ca_address_sk,ca_county,ca_state,ca_country] + 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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + 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_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,cd_demo_sk,cd_dep_count] + CometFilter [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] + 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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 + CometProject [cd_demo_sk,cd_dep_count] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometBroadcastExchange [cd_demo_sk] #5 + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] + CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] + CometBroadcastExchange [d_date_sk] #7 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_item_id] #8 + CometFilter [i_item_sk,i_item_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometHashAggregate [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,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] + CometColumnarExchange [i_item_id,ca_country,ca_state] #9 + CometHashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,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,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country,i_item_sk,i_item_id] + 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_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,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 [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,ca_address_sk,ca_state,ca_country] + 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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + 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_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,cd_demo_sk,cd_dep_count] + CometFilter [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] + 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 + 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 [ca_address_sk,ca_state,ca_country] #10 + CometFilter [ca_address_sk,ca_state,ca_country] + 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 + CometHashAggregate [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,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] + CometColumnarExchange [i_item_id,ca_country] #11 + CometHashAggregate [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,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,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country,i_item_sk,i_item_id] + 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_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,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 [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,ca_address_sk,ca_country] + 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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + 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_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,cd_demo_sk,cd_dep_count] + CometFilter [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] + 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 + 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 [ca_address_sk,ca_country] #12 + CometProject [ca_address_sk,ca_country] + CometFilter [ca_address_sk,ca_state,ca_country] + 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 + CometHashAggregate [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,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] + CometColumnarExchange [i_item_id] #13 + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,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,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk,i_item_id] + 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_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,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 [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,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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + 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_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,cd_demo_sk,cd_dep_count] + CometFilter [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] + 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 + 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 [ca_address_sk] #14 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + 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 + CometHashAggregate [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,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] + CometColumnarExchange #15 + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,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,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,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_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,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 [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,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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + 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_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,cd_demo_sk,cd_dep_count] + CometFilter [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] + 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 + 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 [i_item_sk] #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-spark3_5/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/explain.txt index bfb24de43..e2ecb0b16 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/explain.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject (22) +- * Project (21) +- Window (20) - +- * Sort (19) - +- Exchange (18) - +- * HashAggregate (17) - +- Exchange (16) - +- * ColumnarToRow (15) + +- * ColumnarToRow (19) + +- CometSort (18) + +- CometColumnarExchange (17) + +- CometHashAggregate (16) + +- CometColumnarExchange (15) +- CometHashAggregate (14) +- CometProject (13) +- CometBroadcastHashJoin (12) @@ -92,39 +92,37 @@ Input [6]: [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, 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))] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(16) CometHashAggregate 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] -(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=2] +(17) CometColumnarExchange +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(18) CometSort +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] +Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15], [i_class#9 ASC NULLS FIRST] -(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 +(19) ColumnarToRow [codegen id : 1] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] (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] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] +Arguments: [sum(_w0#15) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#16], [i_class#9] -(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] +(21) Project [codegen id : 2] +Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, ((_w0#15 * 100) / _we0#16) AS revenueratio#17] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, _we0#16] (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] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] +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#17 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/simplified.txt index e460ce5c9..7b17378dc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/simplified.txt @@ -1,38 +1,34 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (4) + WholeStageCodegen (2) Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] InputAdapter Window [_w0,i_class] - WholeStageCodegen (3) - Sort [i_class] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [i_class] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,cs_ext_sales_price] - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometColumnarExchange [i_class] #1 + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum,sum(UnscaledValue(cs_ext_sales_price))] + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,cs_ext_sales_price] + CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + 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-spark3_5/q22/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/explain.txt index 2810779ed..1975ac486 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/explain.txt @@ -1,28 +1,30 @@ == Physical Plan == -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) +TakeOrderedAndProject (26) ++- * HashAggregate (25) + +- * ColumnarToRow (24) + +- CometColumnarExchange (23) + +- RowToColumnar (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 @@ -124,50 +126,56 @@ Functions [1]: [partial_avg(inv_quantity_on_hand#2)] 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] -(22) Exchange +(22) RowToColumnar 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] -(23) HashAggregate [codegen id : 3] +(23) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] + +(24) ColumnarToRow [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] + +(25) 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)#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] -(24) TakeOrderedAndProject +(26) 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 (29) -+- * ColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan parquet spark_catalog.default.date_dim (25) +BroadcastExchange (31) ++- * ColumnarToRow (30) + +- CometProject (29) + +- CometFilter (28) + +- CometScan parquet spark_catalog.default.date_dim (27) -(25) Scan parquet spark_catalog.default.date_dim +(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 -(26) CometFilter +(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)) -(27) CometProject +(29) CometProject Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(28) ColumnarToRow [codegen id : 1] +(30) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(29) BroadcastExchange +(31) BroadcastExchange Input [1]: [d_date_sk#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-v2_7-spark3_5/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt index 0e864ab70..f5b281ac6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt @@ -1,39 +1,41 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] 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 (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 - ColumnarToRow - InputAdapter - CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] - CometProject [inv_item_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.warehouse + ColumnarToRow + InputAdapter + CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 + RowToColumnar + 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 + ColumnarToRow + InputAdapter + CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] + CometProject [inv_item_sk,inv_quantity_on_hand] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + InputAdapter + 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-spark3_5/q22a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/explain.txt index 82e26e3ce..100e35413 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/explain.txt @@ -1,50 +1,50 @@ == Physical Plan == -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) +* ColumnarToRow (46) ++- CometTakeOrderedAndProject (45) + +- CometUnion (44) + :- CometHashAggregate (23) + : +- CometHashAggregate (22) + : +- CometHashAggregate (21) + : +- CometColumnarExchange (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) + :- CometHashAggregate (28) + : +- CometColumnarExchange (27) + : +- CometHashAggregate (26) + : +- CometHashAggregate (25) + : +- ReusedExchange (24) + :- CometHashAggregate (33) + : +- CometColumnarExchange (32) + : +- CometHashAggregate (31) + : +- CometHashAggregate (30) + : +- ReusedExchange (29) + :- CometHashAggregate (38) + : +- CometColumnarExchange (37) + : +- CometHashAggregate (36) + : +- CometHashAggregate (35) + : +- ReusedExchange (34) + +- CometHashAggregate (43) + +- CometColumnarExchange (42) + +- CometHashAggregate (41) + +- CometHashAggregate (40) + +- ReusedExchange (39) (1) Scan parquet spark_catalog.default.inventory @@ -140,151 +140,126 @@ Input [5]: [inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_prod Keys [4]: [i_product_name#12, i_brand#9, i_class#10, i_category#11] Functions [1]: [partial_avg(inv_quantity_on_hand#3)] -(20) ColumnarToRow [codegen id : 1] +(20) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(22) HashAggregate [codegen id : 2] +(21) CometHashAggregate 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)#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] +(22) CometHashAggregate +Input [5]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#16] 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] +Functions [1]: [partial_avg(qoh#16)] -(24) HashAggregate [codegen id : 2] -Input [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#20, count#21] +(23) CometHashAggregate +Input [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#17, count#18] 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#24, i_brand#25, i_class#26, i_category#27, sum#28, count#29] - -(26) HashAggregate [codegen id : 4] -Input [6]: [i_product_name#24, i_brand#25, i_class#26, i_category#27, sum#28, count#29] -Keys [4]: [i_product_name#24, i_brand#25, i_class#26, i_category#27] -Functions [1]: [avg(inv_quantity_on_hand#30)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#30)#16] -Results [4]: [i_product_name#24, i_brand#25, i_class#26, avg(inv_quantity_on_hand#30)#16 AS qoh#31] - -(27) HashAggregate [codegen id : 4] -Input [4]: [i_product_name#24, i_brand#25, i_class#26, qoh#31] -Keys [3]: [i_product_name#24, i_brand#25, i_class#26] -Functions [1]: [partial_avg(qoh#31)] -Aggregate Attributes [2]: [sum#32, count#33] -Results [5]: [i_product_name#24, i_brand#25, i_class#26, sum#34, count#35] - -(28) Exchange -Input [5]: [i_product_name#24, i_brand#25, i_class#26, sum#34, count#35] -Arguments: hashpartitioning(i_product_name#24, i_brand#25, i_class#26, 5), ENSURE_REQUIREMENTS, [plan_id=2] - -(29) HashAggregate [codegen id : 5] -Input [5]: [i_product_name#24, i_brand#25, i_class#26, sum#34, count#35] -Keys [3]: [i_product_name#24, i_brand#25, i_class#26] -Functions [1]: [avg(qoh#31)] -Aggregate Attributes [1]: [avg(qoh#31)#36] -Results [5]: [i_product_name#24, i_brand#25, i_class#26, null AS i_category#37, avg(qoh#31)#36 AS qoh#38] - -(30) ReusedExchange [Reuses operator id: 21] +Functions [1]: [avg(qoh#16)] + +(24) ReusedExchange [Reuses operator id: 20] +Output [6]: [i_product_name#19, i_brand#20, i_class#21, i_category#22, sum#23, count#24] + +(25) CometHashAggregate +Input [6]: [i_product_name#19, i_brand#20, i_class#21, i_category#22, sum#23, count#24] +Keys [4]: [i_product_name#19, i_brand#20, i_class#21, i_category#22] +Functions [1]: [avg(inv_quantity_on_hand#25)] + +(26) CometHashAggregate +Input [4]: [i_product_name#19, i_brand#20, i_class#21, qoh#26] +Keys [3]: [i_product_name#19, i_brand#20, i_class#21] +Functions [1]: [partial_avg(qoh#26)] + +(27) CometColumnarExchange +Input [5]: [i_product_name#19, i_brand#20, i_class#21, sum#27, count#28] +Arguments: hashpartitioning(i_product_name#19, i_brand#20, i_class#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(28) CometHashAggregate +Input [5]: [i_product_name#19, i_brand#20, i_class#21, sum#27, count#28] +Keys [3]: [i_product_name#19, i_brand#20, i_class#21] +Functions [1]: [avg(qoh#26)] + +(29) ReusedExchange [Reuses operator id: 20] +Output [6]: [i_product_name#29, i_brand#30, i_class#31, i_category#32, sum#33, count#34] + +(30) CometHashAggregate +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)] + +(31) CometHashAggregate +Input [3]: [i_product_name#29, i_brand#30, qoh#36] +Keys [2]: [i_product_name#29, i_brand#30] +Functions [1]: [partial_avg(qoh#36)] + +(32) CometColumnarExchange +Input [4]: [i_product_name#29, i_brand#30, sum#37, count#38] +Arguments: hashpartitioning(i_product_name#29, i_brand#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(33) CometHashAggregate +Input [4]: [i_product_name#29, i_brand#30, sum#37, count#38] +Keys [2]: [i_product_name#29, i_brand#30] +Functions [1]: [avg(qoh#36)] + +(34) ReusedExchange [Reuses operator id: 20] Output [6]: [i_product_name#39, i_brand#40, i_class#41, i_category#42, sum#43, count#44] -(31) HashAggregate [codegen id : 7] +(35) CometHashAggregate Input [6]: [i_product_name#39, i_brand#40, i_class#41, i_category#42, sum#43, count#44] Keys [4]: [i_product_name#39, i_brand#40, i_class#41, i_category#42] Functions [1]: [avg(inv_quantity_on_hand#45)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#45)#16] -Results [3]: [i_product_name#39, i_brand#40, avg(inv_quantity_on_hand#45)#16 AS qoh#46] -(32) HashAggregate [codegen id : 7] -Input [3]: [i_product_name#39, i_brand#40, qoh#46] -Keys [2]: [i_product_name#39, i_brand#40] +(36) CometHashAggregate +Input [2]: [i_product_name#39, qoh#46] +Keys [1]: [i_product_name#39] Functions [1]: [partial_avg(qoh#46)] -Aggregate Attributes [2]: [sum#47, count#48] -Results [4]: [i_product_name#39, i_brand#40, sum#49, count#50] -(33) Exchange -Input [4]: [i_product_name#39, i_brand#40, sum#49, count#50] -Arguments: hashpartitioning(i_product_name#39, i_brand#40, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(37) CometColumnarExchange +Input [3]: [i_product_name#39, sum#47, count#48] +Arguments: hashpartitioning(i_product_name#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(34) HashAggregate [codegen id : 8] -Input [4]: [i_product_name#39, i_brand#40, sum#49, count#50] -Keys [2]: [i_product_name#39, i_brand#40] +(38) CometHashAggregate +Input [3]: [i_product_name#39, sum#47, count#48] +Keys [1]: [i_product_name#39] Functions [1]: [avg(qoh#46)] -Aggregate Attributes [1]: [avg(qoh#46)#51] -Results [5]: [i_product_name#39, i_brand#40, null AS i_class#52, null AS i_category#53, avg(qoh#46)#51 AS qoh#54] - -(35) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#55, i_brand#56, i_class#57, i_category#58, sum#59, count#60] - -(36) HashAggregate [codegen id : 10] -Input [6]: [i_product_name#55, i_brand#56, i_class#57, i_category#58, sum#59, count#60] -Keys [4]: [i_product_name#55, i_brand#56, i_class#57, i_category#58] -Functions [1]: [avg(inv_quantity_on_hand#61)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#61)#16] -Results [2]: [i_product_name#55, avg(inv_quantity_on_hand#61)#16 AS qoh#62] - -(37) HashAggregate [codegen id : 10] -Input [2]: [i_product_name#55, qoh#62] -Keys [1]: [i_product_name#55] -Functions [1]: [partial_avg(qoh#62)] -Aggregate Attributes [2]: [sum#63, count#64] -Results [3]: [i_product_name#55, sum#65, count#66] - -(38) Exchange -Input [3]: [i_product_name#55, sum#65, count#66] -Arguments: hashpartitioning(i_product_name#55, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(39) HashAggregate [codegen id : 11] -Input [3]: [i_product_name#55, sum#65, count#66] -Keys [1]: [i_product_name#55] -Functions [1]: [avg(qoh#62)] -Aggregate Attributes [1]: [avg(qoh#62)#67] -Results [5]: [i_product_name#55, null AS i_brand#68, null AS i_class#69, null AS i_category#70, avg(qoh#62)#67 AS qoh#71] - -(40) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#72, i_brand#73, i_class#74, i_category#75, sum#76, count#77] - -(41) HashAggregate [codegen id : 13] -Input [6]: [i_product_name#72, i_brand#73, i_class#74, i_category#75, sum#76, count#77] -Keys [4]: [i_product_name#72, i_brand#73, i_class#74, i_category#75] -Functions [1]: [avg(inv_quantity_on_hand#78)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#78)#16] -Results [1]: [avg(inv_quantity_on_hand#78)#16 AS qoh#79] - -(42) HashAggregate [codegen id : 13] -Input [1]: [qoh#79] + +(39) ReusedExchange [Reuses operator id: 20] +Output [6]: [i_product_name#49, i_brand#50, i_class#51, i_category#52, sum#53, count#54] + +(40) CometHashAggregate +Input [6]: [i_product_name#49, i_brand#50, i_class#51, i_category#52, sum#53, count#54] +Keys [4]: [i_product_name#49, i_brand#50, i_class#51, i_category#52] +Functions [1]: [avg(inv_quantity_on_hand#55)] + +(41) CometHashAggregate +Input [1]: [qoh#56] Keys: [] -Functions [1]: [partial_avg(qoh#79)] -Aggregate Attributes [2]: [sum#80, count#81] -Results [2]: [sum#82, count#83] +Functions [1]: [partial_avg(qoh#56)] -(43) Exchange -Input [2]: [sum#82, count#83] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5] +(42) CometColumnarExchange +Input [2]: [sum#57, count#58] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(44) HashAggregate [codegen id : 14] -Input [2]: [sum#82, count#83] +(43) CometHashAggregate +Input [2]: [sum#57, count#58] Keys: [] -Functions [1]: [avg(qoh#79)] -Aggregate Attributes [1]: [avg(qoh#79)#84] -Results [5]: [null AS i_product_name#85, null AS i_brand#86, null AS i_class#87, null AS i_category#88, avg(qoh#79)#84 AS qoh#89] +Functions [1]: [avg(qoh#56)] + +(44) CometUnion +Child 0 Input [5]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#59] +Child 1 Input [5]: [i_product_name#19, i_brand#20, i_class#21, i_category#60, qoh#61] +Child 2 Input [5]: [i_product_name#29, i_brand#30, i_class#62, i_category#63, qoh#64] +Child 3 Input [5]: [i_product_name#39, i_brand#65, i_class#66, i_category#67, qoh#68] +Child 4 Input [5]: [i_product_name#69, i_brand#70, i_class#71, i_category#72, qoh#73] -(45) Union +(45) CometTakeOrderedAndProject +Input [5]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#59] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#59 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], output=[i_product_name#12,i_brand#9,i_class#10,i_category#11,qoh#59]), [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#59], 100, [qoh#59 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#59] -(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] +(46) ColumnarToRow [codegen id : 1] +Input [5]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#59] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/simplified.txt index 30307e528..d5a409c3e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/simplified.txt @@ -1,74 +1,56 @@ -TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - Union - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count,inv_quantity_on_hand] - CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] - CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] - CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange [w_warehouse_sk] #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] #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 (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] #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 (11) - HashAggregate [i_product_name,sum,count] [avg(qoh),i_brand,i_class,i_category,qoh,sum,count] - InputAdapter - 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 (14) - HashAggregate [sum,count] [avg(qoh),i_product_name,i_brand,i_class,i_category,qoh,sum,count] - InputAdapter - 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 - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_product_name,i_brand,i_class,i_category,qoh] + CometUnion [i_product_name,i_brand,i_class,i_category,qoh] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh,sum,count,avg(qoh)] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count,qoh] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh,sum,count,avg(inv_quantity_on_hand)] + CometColumnarExchange [i_product_name,i_brand,i_class,i_category] #1 + CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count,inv_quantity_on_hand] + CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] + CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] + CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometBroadcastExchange [w_warehouse_sk] #5 + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh,sum,count,avg(qoh)] + CometColumnarExchange [i_product_name,i_brand,i_class] #6 + CometHashAggregate [i_product_name,i_brand,i_class,sum,count,qoh] + CometHashAggregate [i_product_name,i_brand,i_class,qoh,i_category,sum,count,avg(inv_quantity_on_hand)] + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh,sum,count,avg(qoh)] + CometColumnarExchange [i_product_name,i_brand] #7 + CometHashAggregate [i_product_name,i_brand,sum,count,qoh] + CometHashAggregate [i_product_name,i_brand,qoh,i_class,i_category,sum,count,avg(inv_quantity_on_hand)] + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh,sum,count,avg(qoh)] + CometColumnarExchange [i_product_name] #8 + CometHashAggregate [i_product_name,sum,count,qoh] + CometHashAggregate [i_product_name,qoh,i_brand,i_class,i_category,sum,count,avg(inv_quantity_on_hand)] + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh,sum,count,avg(qoh)] + CometColumnarExchange #9 + CometHashAggregate [sum,count,qoh] + CometHashAggregate [qoh,i_product_name,i_brand,i_class,i_category,sum,count,avg(inv_quantity_on_hand)] + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/explain.txt index 0cba9b059..118902e57 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/explain.txt @@ -1,52 +1,54 @@ == Physical Plan == -* Sort (48) -+- Exchange (47) - +- * Filter (46) - +- * HashAggregate (45) - +- Exchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- Exchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Project (14) - : : : : +- * SortMergeJoin Inner (13) - : : : : :- * Sort (6) - : : : : : +- Exchange (5) - : : : : : +- * ColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- * Sort (12) - : : : : +- Exchange (11) - : : : : +- * ColumnarToRow (10) - : : : : +- CometProject (9) - : : : : +- CometFilter (8) - : : : : +- CometScan parquet spark_catalog.default.store_returns (7) - : : : +- BroadcastExchange (19) - : : : +- * ColumnarToRow (18) - : : : +- CometProject (17) - : : : +- CometFilter (16) - : : : +- CometScan parquet spark_catalog.default.store (15) - : : +- BroadcastExchange (25) - : : +- * ColumnarToRow (24) - : : +- CometFilter (23) - : : +- CometScan parquet spark_catalog.default.item (22) - : +- BroadcastExchange (31) - : +- * ColumnarToRow (30) - : +- CometFilter (29) - : +- CometScan parquet spark_catalog.default.customer (28) - +- BroadcastExchange (37) - +- * ColumnarToRow (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.customer_address (34) +* ColumnarToRow (50) ++- CometSort (49) + +- CometColumnarExchange (48) + +- RowToColumnar (47) + +- * Filter (46) + +- * HashAggregate (45) + +- * ColumnarToRow (44) + +- CometColumnarExchange (43) + +- RowToColumnar (42) + +- * HashAggregate (41) + +- * HashAggregate (40) + +- * ColumnarToRow (39) + +- CometColumnarExchange (38) + +- RowToColumnar (37) + +- * HashAggregate (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * ColumnarToRow (29) + : +- CometProject (28) + : +- CometBroadcastHashJoin (27) + : :- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometProject (18) + : : : +- CometBroadcastHashJoin (17) + : : : :- CometProject (12) + : : : : +- CometSortMergeJoin (11) + : : : : :- CometSort (5) + : : : : : +- CometColumnarExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometSort (10) + : : : : +- CometColumnarExchange (9) + : : : : +- CometProject (8) + : : : : +- CometFilter (7) + : : : : +- CometScan parquet spark_catalog.default.store_returns (6) + : : : +- CometBroadcastExchange (16) + : : : +- CometProject (15) + : : : +- CometFilter (14) + : : : +- CometScan parquet spark_catalog.default.store (13) + : : +- CometBroadcastExchange (21) + : : +- CometFilter (20) + : : +- CometScan parquet spark_catalog.default.item (19) + : +- CometBroadcastExchange (26) + : +- CometFilter (25) + : +- CometScan parquet spark_catalog.default.customer (24) + +- BroadcastExchange (33) + +- * ColumnarToRow (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.customer_address (30) (1) Scan parquet spark_catalog.default.store_sales @@ -64,370 +66,384 @@ Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND is Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -(4) ColumnarToRow [codegen id : 1] +(4) CometColumnarExchange Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(5) Exchange +(5) CometSort Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] -(6) Sort [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST], false, 0 - -(7) Scan parquet spark_catalog.default.store_returns +(6) Scan parquet spark_catalog.default.store_returns Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(8) CometFilter +(7) CometFilter Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) -(9) CometProject +(8) CometProject Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] -(10) ColumnarToRow [codegen id : 3] +(9) CometColumnarExchange Input [2]: [sr_item_sk#7, sr_ticket_number#8] +Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(11) Exchange +(10) CometSort Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] -(12) Sort [codegen id : 4] -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST], false, 0 +(11) CometSortMergeJoin +Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Right output [2]: [sr_item_sk#7, sr_ticket_number#8] +Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner -(13) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_ticket_number#4, ss_item_sk#1] -Right keys [2]: [sr_ticket_number#8, sr_item_sk#7] -Join type: Inner -Join condition: None - -(14) Project [codegen id : 9] -Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +(12) CometProject Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -(15) Scan parquet spark_catalog.default.store +(13) Scan parquet spark_catalog.default.store Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(16) CometFilter +(14) CometFilter Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(s_zip#14)) -(17) CometProject +(15) CometProject Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Arguments: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14], [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(18) ColumnarToRow [codegen id : 5] +(16) CometBroadcastExchange Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(19) BroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(17) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +Right output [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight -(20) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#10] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 9] -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] +(18) CometProject Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] -(22) Scan parquet spark_catalog.default.item +(19) Scan parquet spark_catalog.default.item Output [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale ), IsNotNull(i_item_sk)] ReadSchema: struct -(23) CometFilter +(20) CometFilter Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Condition : ((isnotnull(i_color#18) AND (i_color#18 = pale )) AND isnotnull(i_item_sk#15)) -(24) ColumnarToRow [codegen id : 6] +(21) CometBroadcastExchange Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Arguments: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(25) BroadcastExchange -Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(22) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] +Right output [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Arguments: [ss_item_sk#1], [i_item_sk#15], Inner, BuildRight -(26) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#15] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 9] -Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +(23) CometProject Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(28) Scan parquet spark_catalog.default.customer +(24) Scan parquet spark_catalog.default.customer Output [5]: [c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] ReadSchema: struct -(29) CometFilter +(25) CometFilter Input [5]: [c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] Condition : ((isnotnull(c_customer_sk#21) AND isnotnull(c_current_addr_sk#22)) AND isnotnull(c_birth_country#25)) -(30) ColumnarToRow [codegen id : 7] -Input [5]: [c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] - -(31) BroadcastExchange +(26) CometBroadcastExchange Input [5]: [c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +Arguments: [c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] -(32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#21] -Join type: Inner -Join condition: None +(27) CometBroadcastHashJoin +Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Right output [5]: [c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] +Arguments: [ss_customer_sk#2], [c_customer_sk#21], Inner, BuildRight -(33) Project [codegen id : 9] -Output [13]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] +(28) CometProject Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] +Arguments: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25], [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] + +(29) ColumnarToRow [codegen id : 2] +Input [13]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] -(34) Scan parquet spark_catalog.default.customer_address +(30) Scan parquet spark_catalog.default.customer_address Output [4]: [ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct -(35) CometFilter +(31) CometFilter Input [4]: [ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] Condition : ((isnotnull(ca_address_sk#26) AND isnotnull(ca_country#29)) AND isnotnull(ca_zip#28)) -(36) ColumnarToRow [codegen id : 8] +(32) ColumnarToRow [codegen id : 1] Input [4]: [ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] -(37) BroadcastExchange +(33) BroadcastExchange Input [4]: [ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] -Arguments: HashedRelationBroadcastMode(List(input[0, int, false], upper(input[3, string, false]), input[2, string, false]),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(input[0, int, false], upper(input[3, string, false]), input[2, string, false]),false), [plan_id=3] -(38) BroadcastHashJoin [codegen id : 9] +(34) BroadcastHashJoin [codegen id : 2] Left keys [3]: [c_current_addr_sk#22, c_birth_country#25, s_zip#14] Right keys [3]: [ca_address_sk#26, upper(ca_country#29), ca_zip#28] Join type: Inner Join condition: None -(39) Project [codegen id : 9] +(35) Project [codegen id : 2] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, ca_state#27] Input [17]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25, ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] -(40) HashAggregate [codegen id : 9] +(36) HashAggregate [codegen id : 2] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, ca_state#27] Keys [10]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum#30] Results [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#31] -(41) Exchange +(37) RowToColumnar Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#31] -Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(42) HashAggregate [codegen id : 10] +(38) CometColumnarExchange +Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#31] +Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(39) ColumnarToRow [codegen id : 3] +Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#31] + +(40) HashAggregate [codegen id : 3] Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#31] Keys [10]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#32] Results [4]: [c_last_name#24, c_first_name#23, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#32,17,2) AS netpaid#33] -(43) HashAggregate [codegen id : 10] +(41) HashAggregate [codegen id : 3] Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, netpaid#33] Keys [3]: [c_last_name#24, c_first_name#23, s_store_name#11] Functions [1]: [partial_sum(netpaid#33)] Aggregate Attributes [2]: [sum#34, isEmpty#35] Results [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#36, isEmpty#37] -(44) Exchange +(42) RowToColumnar +Input [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#36, isEmpty#37] + +(43) CometColumnarExchange Input [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#36, isEmpty#37] -Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=8] +Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) HashAggregate [codegen id : 11] +(44) ColumnarToRow [codegen id : 4] +Input [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#36, isEmpty#37] + +(45) HashAggregate [codegen id : 4] Input [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#36, isEmpty#37] Keys [3]: [c_last_name#24, c_first_name#23, s_store_name#11] Functions [1]: [sum(netpaid#33)] Aggregate Attributes [1]: [sum(netpaid#33)#38] Results [4]: [c_last_name#24, c_first_name#23, s_store_name#11, sum(netpaid#33)#38 AS paid#39] -(46) Filter [codegen id : 11] +(46) Filter [codegen id : 4] Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] Condition : (isnotnull(paid#39) AND (cast(paid#39 as decimal(33,8)) > cast(Subquery scalar-subquery#40, [id=#41] as decimal(33,8)))) -(47) Exchange +(47) RowToColumnar +Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] + +(48) CometColumnarExchange Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] -Arguments: rangepartitioning(c_last_name#24 ASC NULLS FIRST, c_first_name#23 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Arguments: rangepartitioning(c_last_name#24 ASC NULLS FIRST, c_first_name#23 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(48) Sort [codegen id : 12] +(49) CometSort +Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] +Arguments: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39], [c_last_name#24 ASC NULLS FIRST, c_first_name#23 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST] + +(50) ColumnarToRow [codegen id : 5] Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] -Arguments: [c_last_name#24 ASC NULLS FIRST, c_first_name#23 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#40, [id=#41] -* HashAggregate (75) -+- Exchange (74) - +- * HashAggregate (73) - +- * HashAggregate (72) - +- Exchange (71) - +- * HashAggregate (70) - +- * Project (69) - +- * BroadcastHashJoin Inner BuildRight (68) - :- * Project (66) - : +- * BroadcastHashJoin Inner BuildRight (65) - : :- * Project (63) - : : +- * BroadcastHashJoin Inner BuildRight (62) - : : :- * Project (57) - : : : +- * BroadcastHashJoin Inner BuildRight (56) - : : : :- * Project (54) - : : : : +- * SortMergeJoin Inner (53) - : : : : :- * Sort (50) - : : : : : +- ReusedExchange (49) - : : : : +- * Sort (52) - : : : : +- ReusedExchange (51) - : : : +- ReusedExchange (55) - : : +- BroadcastExchange (61) - : : +- * ColumnarToRow (60) - : : +- CometFilter (59) - : : +- CometScan parquet spark_catalog.default.item (58) - : +- ReusedExchange (64) - +- ReusedExchange (67) - - -(49) ReusedExchange [Reuses operator id: 5] +* HashAggregate (81) ++- * ColumnarToRow (80) + +- CometColumnarExchange (79) + +- RowToColumnar (78) + +- * HashAggregate (77) + +- * HashAggregate (76) + +- * ColumnarToRow (75) + +- CometColumnarExchange (74) + +- RowToColumnar (73) + +- * HashAggregate (72) + +- * Project (71) + +- * BroadcastHashJoin Inner BuildRight (70) + :- * ColumnarToRow (68) + : +- CometProject (67) + : +- CometBroadcastHashJoin (66) + : :- CometProject (64) + : : +- CometBroadcastHashJoin (63) + : : :- CometProject (59) + : : : +- CometBroadcastHashJoin (58) + : : : :- CometProject (56) + : : : : +- CometSortMergeJoin (55) + : : : : :- CometSort (52) + : : : : : +- ReusedExchange (51) + : : : : +- CometSort (54) + : : : : +- ReusedExchange (53) + : : : +- ReusedExchange (57) + : : +- CometBroadcastExchange (62) + : : +- CometFilter (61) + : : +- CometScan parquet spark_catalog.default.item (60) + : +- ReusedExchange (65) + +- ReusedExchange (69) + + +(51) ReusedExchange [Reuses operator id: 4] Output [5]: [ss_item_sk#42, ss_customer_sk#43, ss_store_sk#44, ss_ticket_number#45, ss_net_paid#46] -(50) Sort [codegen id : 2] +(52) CometSort Input [5]: [ss_item_sk#42, ss_customer_sk#43, ss_store_sk#44, ss_ticket_number#45, ss_net_paid#46] -Arguments: [ss_ticket_number#45 ASC NULLS FIRST, ss_item_sk#42 ASC NULLS FIRST], false, 0 +Arguments: [ss_item_sk#42, ss_customer_sk#43, ss_store_sk#44, ss_ticket_number#45, ss_net_paid#46], [ss_ticket_number#45 ASC NULLS FIRST, ss_item_sk#42 ASC NULLS FIRST] -(51) ReusedExchange [Reuses operator id: 11] +(53) ReusedExchange [Reuses operator id: 9] Output [2]: [sr_item_sk#47, sr_ticket_number#48] -(52) Sort [codegen id : 4] +(54) CometSort Input [2]: [sr_item_sk#47, sr_ticket_number#48] -Arguments: [sr_ticket_number#48 ASC NULLS FIRST, sr_item_sk#47 ASC NULLS FIRST], false, 0 +Arguments: [sr_item_sk#47, sr_ticket_number#48], [sr_ticket_number#48 ASC NULLS FIRST, sr_item_sk#47 ASC NULLS FIRST] -(53) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_ticket_number#45, ss_item_sk#42] -Right keys [2]: [sr_ticket_number#48, sr_item_sk#47] -Join type: Inner -Join condition: None +(55) CometSortMergeJoin +Left output [5]: [ss_item_sk#42, ss_customer_sk#43, ss_store_sk#44, ss_ticket_number#45, ss_net_paid#46] +Right output [2]: [sr_item_sk#47, sr_ticket_number#48] +Arguments: [ss_ticket_number#45, ss_item_sk#42], [sr_ticket_number#48, sr_item_sk#47], Inner -(54) Project [codegen id : 9] -Output [4]: [ss_item_sk#42, ss_customer_sk#43, ss_store_sk#44, ss_net_paid#46] +(56) CometProject Input [7]: [ss_item_sk#42, ss_customer_sk#43, ss_store_sk#44, ss_ticket_number#45, ss_net_paid#46, sr_item_sk#47, sr_ticket_number#48] +Arguments: [ss_item_sk#42, ss_customer_sk#43, ss_store_sk#44, ss_net_paid#46], [ss_item_sk#42, ss_customer_sk#43, ss_store_sk#44, ss_net_paid#46] -(55) ReusedExchange [Reuses operator id: 19] +(57) ReusedExchange [Reuses operator id: 16] Output [4]: [s_store_sk#49, s_store_name#50, s_state#51, s_zip#52] -(56) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#44] -Right keys [1]: [s_store_sk#49] -Join type: Inner -Join condition: None +(58) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#42, ss_customer_sk#43, ss_store_sk#44, ss_net_paid#46] +Right output [4]: [s_store_sk#49, s_store_name#50, s_state#51, s_zip#52] +Arguments: [ss_store_sk#44], [s_store_sk#49], Inner, BuildRight -(57) Project [codegen id : 9] -Output [6]: [ss_item_sk#42, ss_customer_sk#43, ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52] +(59) CometProject Input [8]: [ss_item_sk#42, ss_customer_sk#43, ss_store_sk#44, ss_net_paid#46, s_store_sk#49, s_store_name#50, s_state#51, s_zip#52] +Arguments: [ss_item_sk#42, ss_customer_sk#43, ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52], [ss_item_sk#42, ss_customer_sk#43, ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52] -(58) Scan parquet spark_catalog.default.item +(60) Scan parquet spark_catalog.default.item Output [6]: [i_item_sk#53, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(59) CometFilter +(61) CometFilter Input [6]: [i_item_sk#53, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58] Condition : isnotnull(i_item_sk#53) -(60) ColumnarToRow [codegen id : 6] +(62) CometBroadcastExchange Input [6]: [i_item_sk#53, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58] +Arguments: [i_item_sk#53, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58] -(61) BroadcastExchange -Input [6]: [i_item_sk#53, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] +(63) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#42, ss_customer_sk#43, ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52] +Right output [6]: [i_item_sk#53, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58] +Arguments: [ss_item_sk#42], [i_item_sk#53], Inner, BuildRight -(62) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#42] -Right keys [1]: [i_item_sk#53] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 9] -Output [10]: [ss_customer_sk#43, ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58] +(64) CometProject Input [12]: [ss_item_sk#42, ss_customer_sk#43, ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52, i_item_sk#53, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58] +Arguments: [ss_customer_sk#43, ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58], [ss_customer_sk#43, ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58] -(64) ReusedExchange [Reuses operator id: 31] +(65) ReusedExchange [Reuses operator id: 26] Output [5]: [c_customer_sk#59, c_current_addr_sk#60, c_first_name#61, c_last_name#62, c_birth_country#63] -(65) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#43] -Right keys [1]: [c_customer_sk#59] -Join type: Inner -Join condition: None +(66) CometBroadcastHashJoin +Left output [10]: [ss_customer_sk#43, ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58] +Right output [5]: [c_customer_sk#59, c_current_addr_sk#60, c_first_name#61, c_last_name#62, c_birth_country#63] +Arguments: [ss_customer_sk#43], [c_customer_sk#59], Inner, BuildRight -(66) Project [codegen id : 9] -Output [13]: [ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58, c_current_addr_sk#60, c_first_name#61, c_last_name#62, c_birth_country#63] +(67) CometProject Input [15]: [ss_customer_sk#43, ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58, c_customer_sk#59, c_current_addr_sk#60, c_first_name#61, c_last_name#62, c_birth_country#63] +Arguments: [ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58, c_current_addr_sk#60, c_first_name#61, c_last_name#62, c_birth_country#63], [ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58, c_current_addr_sk#60, c_first_name#61, c_last_name#62, c_birth_country#63] -(67) ReusedExchange [Reuses operator id: 37] +(68) ColumnarToRow [codegen id : 2] +Input [13]: [ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58, c_current_addr_sk#60, c_first_name#61, c_last_name#62, c_birth_country#63] + +(69) ReusedExchange [Reuses operator id: 33] Output [4]: [ca_address_sk#64, ca_state#65, ca_zip#66, ca_country#67] -(68) BroadcastHashJoin [codegen id : 9] +(70) BroadcastHashJoin [codegen id : 2] Left keys [3]: [c_current_addr_sk#60, c_birth_country#63, s_zip#52] Right keys [3]: [ca_address_sk#64, upper(ca_country#67), ca_zip#66] Join type: Inner Join condition: None -(69) Project [codegen id : 9] +(71) Project [codegen id : 2] Output [11]: [ss_net_paid#46, s_store_name#50, s_state#51, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58, c_first_name#61, c_last_name#62, ca_state#65] Input [17]: [ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58, c_current_addr_sk#60, c_first_name#61, c_last_name#62, c_birth_country#63, ca_address_sk#64, ca_state#65, ca_zip#66, ca_country#67] -(70) HashAggregate [codegen id : 9] +(72) HashAggregate [codegen id : 2] Input [11]: [ss_net_paid#46, s_store_name#50, s_state#51, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58, c_first_name#61, c_last_name#62, ca_state#65] Keys [10]: [c_last_name#62, c_first_name#61, s_store_name#50, ca_state#65, s_state#51, i_color#56, i_current_price#54, i_manager_id#58, i_units#57, i_size#55] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#46))] Aggregate Attributes [1]: [sum#68] Results [11]: [c_last_name#62, c_first_name#61, s_store_name#50, ca_state#65, s_state#51, i_color#56, i_current_price#54, i_manager_id#58, i_units#57, i_size#55, sum#69] -(71) Exchange +(73) RowToColumnar +Input [11]: [c_last_name#62, c_first_name#61, s_store_name#50, ca_state#65, s_state#51, i_color#56, i_current_price#54, i_manager_id#58, i_units#57, i_size#55, sum#69] + +(74) CometColumnarExchange Input [11]: [c_last_name#62, c_first_name#61, s_store_name#50, ca_state#65, s_state#51, i_color#56, i_current_price#54, i_manager_id#58, i_units#57, i_size#55, sum#69] -Arguments: hashpartitioning(c_last_name#62, c_first_name#61, s_store_name#50, ca_state#65, s_state#51, i_color#56, i_current_price#54, i_manager_id#58, i_units#57, i_size#55, 5), ENSURE_REQUIREMENTS, [plan_id=11] +Arguments: hashpartitioning(c_last_name#62, c_first_name#61, s_store_name#50, ca_state#65, s_state#51, i_color#56, i_current_price#54, i_manager_id#58, i_units#57, i_size#55, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(72) HashAggregate [codegen id : 10] +(75) ColumnarToRow [codegen id : 3] +Input [11]: [c_last_name#62, c_first_name#61, s_store_name#50, ca_state#65, s_state#51, i_color#56, i_current_price#54, i_manager_id#58, i_units#57, i_size#55, sum#69] + +(76) HashAggregate [codegen id : 3] Input [11]: [c_last_name#62, c_first_name#61, s_store_name#50, ca_state#65, s_state#51, i_color#56, i_current_price#54, i_manager_id#58, i_units#57, i_size#55, sum#69] Keys [10]: [c_last_name#62, c_first_name#61, s_store_name#50, ca_state#65, s_state#51, i_color#56, i_current_price#54, i_manager_id#58, i_units#57, i_size#55] Functions [1]: [sum(UnscaledValue(ss_net_paid#46))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#46))#32] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#46))#32,17,2) AS netpaid#70] -(73) HashAggregate [codegen id : 10] +(77) HashAggregate [codegen id : 3] Input [1]: [netpaid#70] Keys: [] Functions [1]: [partial_avg(netpaid#70)] Aggregate Attributes [2]: [sum#71, count#72] Results [2]: [sum#73, count#74] -(74) Exchange +(78) RowToColumnar +Input [2]: [sum#73, count#74] + +(79) CometColumnarExchange +Input [2]: [sum#73, count#74] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(80) ColumnarToRow [codegen id : 4] Input [2]: [sum#73, count#74] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] -(75) HashAggregate [codegen id : 11] +(81) HashAggregate [codegen id : 4] Input [2]: [sum#73, count#74] Keys: [] Functions [1]: [avg(netpaid#70)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/simplified.txt index 5b658f1d7..42c283018 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/simplified.txt @@ -1,122 +1,100 @@ -WholeStageCodegen (12) - Sort [c_last_name,c_first_name,s_store_name] +WholeStageCodegen (5) + ColumnarToRow InputAdapter - Exchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (11) - Filter [paid] - Subquery #1 - WholeStageCodegen (11) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - InputAdapter - Exchange #10 - WholeStageCodegen (10) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - InputAdapter - Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 - WholeStageCodegen (9) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - WholeStageCodegen (2) - Sort [ss_ticket_number,ss_item_sk] - InputAdapter - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #4 - InputAdapter - WholeStageCodegen (4) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - ReusedExchange [sr_item_sk,sr_ticket_number] #5 - InputAdapter - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #6 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - InputAdapter - ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 - InputAdapter - ReusedExchange [ca_address_sk,ca_state,ca_zip,ca_country] #9 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - InputAdapter - Exchange [c_last_name,c_first_name,s_store_name] #2 - WholeStageCodegen (10) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + CometSort [c_last_name,c_first_name,s_store_name,paid] + CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 + RowToColumnar + WholeStageCodegen (4) + Filter [paid] + Subquery #1 + WholeStageCodegen (4) + HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] + ColumnarToRow InputAdapter - Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 - WholeStageCodegen (9) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - WholeStageCodegen (2) - Sort [ss_ticket_number,ss_item_sk] - InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + CometColumnarExchange #10 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [netpaid] [sum,count,sum,count] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] + ColumnarToRow + InputAdapter + CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] + CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] + ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #4 + CometSort [sr_item_sk,sr_ticket_number] + ReusedExchange [sr_item_sk,sr_ticket_number] #5 + ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #6 + CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #12 + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 InputAdapter - WholeStageCodegen (4) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - Exchange [sr_ticket_number,sr_item_sk] #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + ReusedExchange [ca_address_sk,ca_state,ca_zip,ca_country] #9 + HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [c_last_name,c_first_name,s_store_name] #2 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] + ColumnarToRow + InputAdapter + CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] + CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #4 + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + CometSort [sr_item_sk,sr_ticket_number] + CometColumnarExchange [sr_ticket_number,sr_item_sk] #5 + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #6 + CometProject [s_store_sk,s_store_name,s_state,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #7 + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) + BroadcastExchange #9 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] + CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/explain.txt index 88f215800..33a1e3542 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/explain.txt @@ -1,75 +1,73 @@ == Physical Plan == -TakeOrderedAndProject (71) -+- Union (70) - :- * HashAggregate (28) - : +- Exchange (27) - : +- * 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 (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) +* ColumnarToRow (69) ++- CometTakeOrderedAndProject (68) + +- CometUnion (67) + :- CometHashAggregate (27) + : +- CometColumnarExchange (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) + :- CometHashAggregate (47) + : +- CometColumnarExchange (46) + : +- CometHashAggregate (45) + : +- CometProject (44) + : +- CometBroadcastHashJoin (43) + : :- CometProject (41) + : : +- CometBroadcastHashJoin (40) + : : :- CometProject (35) + : : : +- CometBroadcastHashJoin (34) + : : : :- CometProject (32) + : : : : +- CometBroadcastHashJoin (31) + : : : : :- CometFilter (29) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (28) + : : : : +- ReusedExchange (30) + : : : +- ReusedExchange (33) + : : +- CometBroadcastExchange (39) + : : +- CometProject (38) + : : +- CometFilter (37) + : : +- CometScan parquet spark_catalog.default.store (36) + : +- ReusedExchange (42) + +- CometHashAggregate (66) + +- CometColumnarExchange (65) + +- CometHashAggregate (64) + +- CometProject (63) + +- CometBroadcastHashJoin (62) + :- CometProject (58) + : +- CometBroadcastHashJoin (57) + : :- CometProject (55) + : : +- CometBroadcastHashJoin (54) + : : :- CometProject (52) + : : : +- CometBroadcastHashJoin (51) + : : : :- CometFilter (49) + : : : : +- CometScan parquet spark_catalog.default.store_sales (48) + : : : +- ReusedExchange (50) + : : +- ReusedExchange (53) + : +- ReusedExchange (56) + +- CometBroadcastExchange (61) + +- CometFilter (60) + +- CometScan parquet spark_catalog.default.item (59) (1) Scan parquet spark_catalog.default.store_sales @@ -193,246 +191,237 @@ 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) ColumnarToRow [codegen id : 1] +(26) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(27) Exchange -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 : 2] +(27) CometHashAggregate 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#41, ss_cdemo_sk#42, ss_store_sk#43, ss_quantity#44, ss_list_price#45, ss_sales_price#46, ss_coupon_amt#47, ss_sold_date_sk#48] +(28) Scan parquet spark_catalog.default.store_sales +Output [8]: [ss_item_sk#32, ss_cdemo_sk#33, ss_store_sk#34, ss_quantity#35, ss_list_price#36, ss_sales_price#37, ss_coupon_amt#38, ss_sold_date_sk#39] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#48), dynamicpruningexpression(ss_sold_date_sk#48 IN dynamicpruning#49)] +PartitionFilters: [isnotnull(ss_sold_date_sk#39), dynamicpruningexpression(ss_sold_date_sk#39 IN dynamicpruning#40)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(30) CometFilter -Input [8]: [ss_item_sk#41, ss_cdemo_sk#42, ss_store_sk#43, ss_quantity#44, ss_list_price#45, ss_sales_price#46, ss_coupon_amt#47, ss_sold_date_sk#48] -Condition : ((isnotnull(ss_cdemo_sk#42) AND isnotnull(ss_store_sk#43)) AND isnotnull(ss_item_sk#41)) +(29) CometFilter +Input [8]: [ss_item_sk#32, ss_cdemo_sk#33, ss_store_sk#34, ss_quantity#35, ss_list_price#36, ss_sales_price#37, ss_coupon_amt#38, ss_sold_date_sk#39] +Condition : ((isnotnull(ss_cdemo_sk#33) AND isnotnull(ss_store_sk#34)) AND isnotnull(ss_item_sk#32)) -(31) ReusedExchange [Reuses operator id: 6] -Output [1]: [cd_demo_sk#50] +(30) ReusedExchange [Reuses operator id: 6] +Output [1]: [cd_demo_sk#41] -(32) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#41, ss_cdemo_sk#42, ss_store_sk#43, ss_quantity#44, ss_list_price#45, ss_sales_price#46, ss_coupon_amt#47, ss_sold_date_sk#48] -Right output [1]: [cd_demo_sk#50] -Arguments: [ss_cdemo_sk#42], [cd_demo_sk#50], Inner, BuildRight +(31) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#32, ss_cdemo_sk#33, ss_store_sk#34, ss_quantity#35, ss_list_price#36, ss_sales_price#37, ss_coupon_amt#38, ss_sold_date_sk#39] +Right output [1]: [cd_demo_sk#41] +Arguments: [ss_cdemo_sk#33], [cd_demo_sk#41], Inner, BuildRight -(33) CometProject -Input [9]: [ss_item_sk#41, ss_cdemo_sk#42, ss_store_sk#43, ss_quantity#44, ss_list_price#45, ss_sales_price#46, ss_coupon_amt#47, ss_sold_date_sk#48, cd_demo_sk#50] -Arguments: [ss_item_sk#41, ss_store_sk#43, ss_quantity#44, ss_list_price#45, ss_sales_price#46, ss_coupon_amt#47, ss_sold_date_sk#48], [ss_item_sk#41, ss_store_sk#43, ss_quantity#44, ss_list_price#45, ss_sales_price#46, ss_coupon_amt#47, ss_sold_date_sk#48] +(32) CometProject +Input [9]: [ss_item_sk#32, ss_cdemo_sk#33, ss_store_sk#34, ss_quantity#35, ss_list_price#36, ss_sales_price#37, ss_coupon_amt#38, ss_sold_date_sk#39, cd_demo_sk#41] +Arguments: [ss_item_sk#32, ss_store_sk#34, ss_quantity#35, ss_list_price#36, ss_sales_price#37, ss_coupon_amt#38, ss_sold_date_sk#39], [ss_item_sk#32, ss_store_sk#34, ss_quantity#35, ss_list_price#36, ss_sales_price#37, ss_coupon_amt#38, ss_sold_date_sk#39] -(34) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#51] +(33) ReusedExchange [Reuses operator id: 12] +Output [1]: [d_date_sk#42] -(35) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#41, ss_store_sk#43, ss_quantity#44, ss_list_price#45, ss_sales_price#46, ss_coupon_amt#47, ss_sold_date_sk#48] -Right output [1]: [d_date_sk#51] -Arguments: [ss_sold_date_sk#48], [d_date_sk#51], Inner, BuildRight +(34) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#32, ss_store_sk#34, ss_quantity#35, ss_list_price#36, ss_sales_price#37, ss_coupon_amt#38, ss_sold_date_sk#39] +Right output [1]: [d_date_sk#42] +Arguments: [ss_sold_date_sk#39], [d_date_sk#42], Inner, BuildRight -(36) CometProject -Input [8]: [ss_item_sk#41, ss_store_sk#43, ss_quantity#44, ss_list_price#45, ss_sales_price#46, ss_coupon_amt#47, ss_sold_date_sk#48, d_date_sk#51] -Arguments: [ss_item_sk#41, ss_store_sk#43, ss_quantity#44, ss_list_price#45, ss_sales_price#46, ss_coupon_amt#47], [ss_item_sk#41, ss_store_sk#43, ss_quantity#44, ss_list_price#45, ss_sales_price#46, ss_coupon_amt#47] +(35) CometProject +Input [8]: [ss_item_sk#32, ss_store_sk#34, ss_quantity#35, ss_list_price#36, ss_sales_price#37, ss_coupon_amt#38, ss_sold_date_sk#39, d_date_sk#42] +Arguments: [ss_item_sk#32, ss_store_sk#34, ss_quantity#35, ss_list_price#36, ss_sales_price#37, ss_coupon_amt#38], [ss_item_sk#32, ss_store_sk#34, ss_quantity#35, ss_list_price#36, ss_sales_price#37, ss_coupon_amt#38] -(37) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#52, s_state#53] +(36) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#43, s_state#44] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct -(38) CometFilter -Input [2]: [s_store_sk#52, s_state#53] -Condition : ((isnotnull(s_state#53) AND (s_state#53 = TN)) AND isnotnull(s_store_sk#52)) +(37) CometFilter +Input [2]: [s_store_sk#43, s_state#44] +Condition : ((isnotnull(s_state#44) AND (s_state#44 = TN)) AND isnotnull(s_store_sk#43)) -(39) CometProject -Input [2]: [s_store_sk#52, s_state#53] -Arguments: [s_store_sk#52], [s_store_sk#52] +(38) CometProject +Input [2]: [s_store_sk#43, s_state#44] +Arguments: [s_store_sk#43], [s_store_sk#43] -(40) CometBroadcastExchange -Input [1]: [s_store_sk#52] -Arguments: [s_store_sk#52] +(39) CometBroadcastExchange +Input [1]: [s_store_sk#43] +Arguments: [s_store_sk#43] -(41) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#41, ss_store_sk#43, ss_quantity#44, ss_list_price#45, ss_sales_price#46, ss_coupon_amt#47] -Right output [1]: [s_store_sk#52] -Arguments: [ss_store_sk#43], [s_store_sk#52], Inner, BuildRight +(40) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#32, ss_store_sk#34, ss_quantity#35, ss_list_price#36, ss_sales_price#37, ss_coupon_amt#38] +Right output [1]: [s_store_sk#43] +Arguments: [ss_store_sk#34], [s_store_sk#43], Inner, BuildRight -(42) CometProject -Input [7]: [ss_item_sk#41, ss_store_sk#43, ss_quantity#44, ss_list_price#45, ss_sales_price#46, ss_coupon_amt#47, s_store_sk#52] -Arguments: [ss_item_sk#41, ss_quantity#44, ss_list_price#45, ss_sales_price#46, ss_coupon_amt#47], [ss_item_sk#41, ss_quantity#44, ss_list_price#45, ss_sales_price#46, ss_coupon_amt#47] +(41) CometProject +Input [7]: [ss_item_sk#32, ss_store_sk#34, ss_quantity#35, ss_list_price#36, ss_sales_price#37, ss_coupon_amt#38, s_store_sk#43] +Arguments: [ss_item_sk#32, ss_quantity#35, ss_list_price#36, ss_sales_price#37, ss_coupon_amt#38], [ss_item_sk#32, ss_quantity#35, ss_list_price#36, ss_sales_price#37, ss_coupon_amt#38] -(43) ReusedExchange [Reuses operator id: 22] -Output [2]: [i_item_sk#54, i_item_id#55] +(42) ReusedExchange [Reuses operator id: 22] +Output [2]: [i_item_sk#45, i_item_id#46] -(44) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#41, ss_quantity#44, ss_list_price#45, ss_sales_price#46, ss_coupon_amt#47] -Right output [2]: [i_item_sk#54, i_item_id#55] -Arguments: [ss_item_sk#41], [i_item_sk#54], Inner, BuildRight +(43) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#32, ss_quantity#35, ss_list_price#36, ss_sales_price#37, ss_coupon_amt#38] +Right output [2]: [i_item_sk#45, i_item_id#46] +Arguments: [ss_item_sk#32], [i_item_sk#45], Inner, BuildRight -(45) CometProject -Input [7]: [ss_item_sk#41, ss_quantity#44, ss_list_price#45, ss_sales_price#46, ss_coupon_amt#47, i_item_sk#54, i_item_id#55] -Arguments: [i_item_id#55, agg1#56, agg2#57, agg3#58, agg4#59], [i_item_id#55, ss_quantity#44 AS agg1#56, ss_list_price#45 AS agg2#57, ss_coupon_amt#47 AS agg3#58, ss_sales_price#46 AS agg4#59] +(44) CometProject +Input [7]: [ss_item_sk#32, ss_quantity#35, ss_list_price#36, ss_sales_price#37, ss_coupon_amt#38, i_item_sk#45, i_item_id#46] +Arguments: [i_item_id#46, agg1#47, agg2#48, agg3#49, agg4#50], [i_item_id#46, ss_quantity#35 AS agg1#47, ss_list_price#36 AS agg2#48, ss_coupon_amt#38 AS agg3#49, ss_sales_price#37 AS agg4#50] -(46) CometHashAggregate -Input [5]: [i_item_id#55, agg1#56, agg2#57, agg3#58, agg4#59] -Keys [1]: [i_item_id#55] -Functions [4]: [partial_avg(agg1#56), partial_avg(UnscaledValue(agg2#57)), partial_avg(UnscaledValue(agg3#58)), partial_avg(UnscaledValue(agg4#59))] +(45) CometHashAggregate +Input [5]: [i_item_id#46, agg1#47, agg2#48, agg3#49, agg4#50] +Keys [1]: [i_item_id#46] +Functions [4]: [partial_avg(agg1#47), partial_avg(UnscaledValue(agg2#48)), partial_avg(UnscaledValue(agg3#49)), partial_avg(UnscaledValue(agg4#50))] -(47) ColumnarToRow [codegen id : 3] -Input [9]: [i_item_id#55, sum#60, count#61, sum#62, count#63, sum#64, count#65, sum#66, count#67] +(46) CometColumnarExchange +Input [9]: [i_item_id#46, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58] +Arguments: hashpartitioning(i_item_id#46, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(48) Exchange -Input [9]: [i_item_id#55, sum#60, count#61, sum#62, count#63, sum#64, count#65, sum#66, count#67] -Arguments: hashpartitioning(i_item_id#55, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(47) CometHashAggregate +Input [9]: [i_item_id#46, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58] +Keys [1]: [i_item_id#46] +Functions [4]: [avg(agg1#47), avg(UnscaledValue(agg2#48)), avg(UnscaledValue(agg3#49)), avg(UnscaledValue(agg4#50))] -(49) HashAggregate [codegen id : 4] -Input [9]: [i_item_id#55, sum#60, count#61, sum#62, count#63, sum#64, count#65, sum#66, count#67] -Keys [1]: [i_item_id#55] -Functions [4]: [avg(agg1#56), avg(UnscaledValue(agg2#57)), avg(UnscaledValue(agg3#58)), avg(UnscaledValue(agg4#59))] -Aggregate Attributes [4]: [avg(agg1#56)#68, avg(UnscaledValue(agg2#57))#69, avg(UnscaledValue(agg3#58))#70, avg(UnscaledValue(agg4#59))#71] -Results [7]: [i_item_id#55, null AS s_state#72, 1 AS g_state#73, avg(agg1#56)#68 AS agg1#74, cast((avg(UnscaledValue(agg2#57))#69 / 100.0) as decimal(11,6)) AS agg2#75, cast((avg(UnscaledValue(agg3#58))#70 / 100.0) as decimal(11,6)) AS agg3#76, cast((avg(UnscaledValue(agg4#59))#71 / 100.0) as decimal(11,6)) AS agg4#77] - -(50) Scan parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#78, ss_cdemo_sk#79, ss_store_sk#80, ss_quantity#81, ss_list_price#82, ss_sales_price#83, ss_coupon_amt#84, ss_sold_date_sk#85] +(48) Scan parquet spark_catalog.default.store_sales +Output [8]: [ss_item_sk#59, ss_cdemo_sk#60, ss_store_sk#61, ss_quantity#62, ss_list_price#63, ss_sales_price#64, ss_coupon_amt#65, ss_sold_date_sk#66] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#85), dynamicpruningexpression(ss_sold_date_sk#85 IN dynamicpruning#86)] +PartitionFilters: [isnotnull(ss_sold_date_sk#66), dynamicpruningexpression(ss_sold_date_sk#66 IN dynamicpruning#67)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(51) CometFilter -Input [8]: [ss_item_sk#78, ss_cdemo_sk#79, ss_store_sk#80, ss_quantity#81, ss_list_price#82, ss_sales_price#83, ss_coupon_amt#84, ss_sold_date_sk#85] -Condition : ((isnotnull(ss_cdemo_sk#79) AND isnotnull(ss_store_sk#80)) AND isnotnull(ss_item_sk#78)) +(49) CometFilter +Input [8]: [ss_item_sk#59, ss_cdemo_sk#60, ss_store_sk#61, ss_quantity#62, ss_list_price#63, ss_sales_price#64, ss_coupon_amt#65, ss_sold_date_sk#66] +Condition : ((isnotnull(ss_cdemo_sk#60) AND isnotnull(ss_store_sk#61)) AND isnotnull(ss_item_sk#59)) -(52) ReusedExchange [Reuses operator id: 6] -Output [1]: [cd_demo_sk#87] +(50) ReusedExchange [Reuses operator id: 6] +Output [1]: [cd_demo_sk#68] -(53) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#78, ss_cdemo_sk#79, ss_store_sk#80, ss_quantity#81, ss_list_price#82, ss_sales_price#83, ss_coupon_amt#84, ss_sold_date_sk#85] -Right output [1]: [cd_demo_sk#87] -Arguments: [ss_cdemo_sk#79], [cd_demo_sk#87], Inner, BuildRight +(51) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#59, ss_cdemo_sk#60, ss_store_sk#61, ss_quantity#62, ss_list_price#63, ss_sales_price#64, ss_coupon_amt#65, ss_sold_date_sk#66] +Right output [1]: [cd_demo_sk#68] +Arguments: [ss_cdemo_sk#60], [cd_demo_sk#68], Inner, BuildRight -(54) CometProject -Input [9]: [ss_item_sk#78, ss_cdemo_sk#79, ss_store_sk#80, ss_quantity#81, ss_list_price#82, ss_sales_price#83, ss_coupon_amt#84, ss_sold_date_sk#85, cd_demo_sk#87] -Arguments: [ss_item_sk#78, ss_store_sk#80, ss_quantity#81, ss_list_price#82, ss_sales_price#83, ss_coupon_amt#84, ss_sold_date_sk#85], [ss_item_sk#78, ss_store_sk#80, ss_quantity#81, ss_list_price#82, ss_sales_price#83, ss_coupon_amt#84, ss_sold_date_sk#85] +(52) CometProject +Input [9]: [ss_item_sk#59, ss_cdemo_sk#60, ss_store_sk#61, ss_quantity#62, ss_list_price#63, ss_sales_price#64, ss_coupon_amt#65, ss_sold_date_sk#66, cd_demo_sk#68] +Arguments: [ss_item_sk#59, ss_store_sk#61, ss_quantity#62, ss_list_price#63, ss_sales_price#64, ss_coupon_amt#65, ss_sold_date_sk#66], [ss_item_sk#59, ss_store_sk#61, ss_quantity#62, ss_list_price#63, ss_sales_price#64, ss_coupon_amt#65, ss_sold_date_sk#66] -(55) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#88] +(53) ReusedExchange [Reuses operator id: 12] +Output [1]: [d_date_sk#69] -(56) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#78, ss_store_sk#80, ss_quantity#81, ss_list_price#82, ss_sales_price#83, ss_coupon_amt#84, ss_sold_date_sk#85] -Right output [1]: [d_date_sk#88] -Arguments: [ss_sold_date_sk#85], [d_date_sk#88], Inner, BuildRight +(54) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#59, ss_store_sk#61, ss_quantity#62, ss_list_price#63, ss_sales_price#64, ss_coupon_amt#65, ss_sold_date_sk#66] +Right output [1]: [d_date_sk#69] +Arguments: [ss_sold_date_sk#66], [d_date_sk#69], Inner, BuildRight -(57) CometProject -Input [8]: [ss_item_sk#78, ss_store_sk#80, ss_quantity#81, ss_list_price#82, ss_sales_price#83, ss_coupon_amt#84, ss_sold_date_sk#85, d_date_sk#88] -Arguments: [ss_item_sk#78, ss_store_sk#80, ss_quantity#81, ss_list_price#82, ss_sales_price#83, ss_coupon_amt#84], [ss_item_sk#78, ss_store_sk#80, ss_quantity#81, ss_list_price#82, ss_sales_price#83, ss_coupon_amt#84] +(55) CometProject +Input [8]: [ss_item_sk#59, ss_store_sk#61, ss_quantity#62, ss_list_price#63, ss_sales_price#64, ss_coupon_amt#65, ss_sold_date_sk#66, d_date_sk#69] +Arguments: [ss_item_sk#59, ss_store_sk#61, ss_quantity#62, ss_list_price#63, ss_sales_price#64, ss_coupon_amt#65], [ss_item_sk#59, ss_store_sk#61, ss_quantity#62, ss_list_price#63, ss_sales_price#64, ss_coupon_amt#65] -(58) ReusedExchange [Reuses operator id: 40] -Output [1]: [s_store_sk#89] +(56) ReusedExchange [Reuses operator id: 39] +Output [1]: [s_store_sk#70] -(59) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#78, ss_store_sk#80, ss_quantity#81, ss_list_price#82, ss_sales_price#83, ss_coupon_amt#84] -Right output [1]: [s_store_sk#89] -Arguments: [ss_store_sk#80], [s_store_sk#89], Inner, BuildRight +(57) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#59, ss_store_sk#61, ss_quantity#62, ss_list_price#63, ss_sales_price#64, ss_coupon_amt#65] +Right output [1]: [s_store_sk#70] +Arguments: [ss_store_sk#61], [s_store_sk#70], Inner, BuildRight -(60) CometProject -Input [7]: [ss_item_sk#78, ss_store_sk#80, ss_quantity#81, ss_list_price#82, ss_sales_price#83, ss_coupon_amt#84, s_store_sk#89] -Arguments: [ss_item_sk#78, ss_quantity#81, ss_list_price#82, ss_sales_price#83, ss_coupon_amt#84], [ss_item_sk#78, ss_quantity#81, ss_list_price#82, ss_sales_price#83, ss_coupon_amt#84] +(58) CometProject +Input [7]: [ss_item_sk#59, ss_store_sk#61, ss_quantity#62, ss_list_price#63, ss_sales_price#64, ss_coupon_amt#65, s_store_sk#70] +Arguments: [ss_item_sk#59, ss_quantity#62, ss_list_price#63, ss_sales_price#64, ss_coupon_amt#65], [ss_item_sk#59, ss_quantity#62, ss_list_price#63, ss_sales_price#64, ss_coupon_amt#65] -(61) Scan parquet spark_catalog.default.item -Output [1]: [i_item_sk#90] +(59) Scan parquet spark_catalog.default.item +Output [1]: [i_item_sk#71] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(62) CometFilter -Input [1]: [i_item_sk#90] -Condition : isnotnull(i_item_sk#90) +(60) CometFilter +Input [1]: [i_item_sk#71] +Condition : isnotnull(i_item_sk#71) -(63) CometBroadcastExchange -Input [1]: [i_item_sk#90] -Arguments: [i_item_sk#90] +(61) CometBroadcastExchange +Input [1]: [i_item_sk#71] +Arguments: [i_item_sk#71] -(64) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#78, ss_quantity#81, ss_list_price#82, ss_sales_price#83, ss_coupon_amt#84] -Right output [1]: [i_item_sk#90] -Arguments: [ss_item_sk#78], [i_item_sk#90], Inner, BuildRight +(62) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#59, ss_quantity#62, ss_list_price#63, ss_sales_price#64, ss_coupon_amt#65] +Right output [1]: [i_item_sk#71] +Arguments: [ss_item_sk#59], [i_item_sk#71], Inner, BuildRight -(65) CometProject -Input [6]: [ss_item_sk#78, ss_quantity#81, ss_list_price#82, ss_sales_price#83, ss_coupon_amt#84, i_item_sk#90] -Arguments: [agg1#91, agg2#92, agg3#93, agg4#94], [ss_quantity#81 AS agg1#91, ss_list_price#82 AS agg2#92, ss_coupon_amt#84 AS agg3#93, ss_sales_price#83 AS agg4#94] +(63) CometProject +Input [6]: [ss_item_sk#59, ss_quantity#62, ss_list_price#63, ss_sales_price#64, ss_coupon_amt#65, i_item_sk#71] +Arguments: [agg1#72, agg2#73, agg3#74, agg4#75], [ss_quantity#62 AS agg1#72, ss_list_price#63 AS agg2#73, ss_coupon_amt#65 AS agg3#74, ss_sales_price#64 AS agg4#75] -(66) CometHashAggregate -Input [4]: [agg1#91, agg2#92, agg3#93, agg4#94] +(64) CometHashAggregate +Input [4]: [agg1#72, agg2#73, agg3#74, agg4#75] Keys: [] -Functions [4]: [partial_avg(agg1#91), partial_avg(UnscaledValue(agg2#92)), partial_avg(UnscaledValue(agg3#93)), partial_avg(UnscaledValue(agg4#94))] +Functions [4]: [partial_avg(agg1#72), partial_avg(UnscaledValue(agg2#73)), partial_avg(UnscaledValue(agg3#74)), partial_avg(UnscaledValue(agg4#75))] -(67) ColumnarToRow [codegen id : 5] -Input [8]: [sum#95, count#96, sum#97, count#98, sum#99, count#100, sum#101, count#102] +(65) CometColumnarExchange +Input [8]: [sum#76, count#77, sum#78, count#79, sum#80, count#81, sum#82, count#83] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(68) Exchange -Input [8]: [sum#95, count#96, sum#97, count#98, sum#99, count#100, sum#101, count#102] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] - -(69) HashAggregate [codegen id : 6] -Input [8]: [sum#95, count#96, sum#97, count#98, sum#99, count#100, sum#101, count#102] +(66) CometHashAggregate +Input [8]: [sum#76, count#77, sum#78, count#79, sum#80, count#81, sum#82, count#83] Keys: [] -Functions [4]: [avg(agg1#91), avg(UnscaledValue(agg2#92)), avg(UnscaledValue(agg3#93)), avg(UnscaledValue(agg4#94))] -Aggregate Attributes [4]: [avg(agg1#91)#103, avg(UnscaledValue(agg2#92))#104, avg(UnscaledValue(agg3#93))#105, avg(UnscaledValue(agg4#94))#106] -Results [7]: [null AS i_item_id#107, null AS s_state#108, 1 AS g_state#109, avg(agg1#91)#103 AS agg1#110, cast((avg(UnscaledValue(agg2#92))#104 / 100.0) as decimal(11,6)) AS agg2#111, cast((avg(UnscaledValue(agg3#93))#105 / 100.0) as decimal(11,6)) AS agg3#112, cast((avg(UnscaledValue(agg4#94))#106 / 100.0) as decimal(11,6)) AS agg4#113] +Functions [4]: [avg(agg1#72), avg(UnscaledValue(agg2#73)), avg(UnscaledValue(agg3#74)), avg(UnscaledValue(agg4#75))] + +(67) CometUnion +Child 0 Input [7]: [i_item_id#19, s_state#17, g_state#84, agg1#85, agg2#86, agg3#87, agg4#88] +Child 1 Input [7]: [i_item_id#46, s_state#89, g_state#90, agg1#91, agg2#92, agg3#93, agg4#94] +Child 2 Input [7]: [i_item_id#95, s_state#96, g_state#97, agg1#98, agg2#99, agg3#100, agg4#101] -(70) Union +(68) CometTakeOrderedAndProject +Input [7]: [i_item_id#19, s_state#17, g_state#84, agg1#85, agg2#86, agg3#87, agg4#88] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#19 ASC NULLS FIRST,s_state#17 ASC NULLS FIRST], output=[i_item_id#19,s_state#17,g_state#84,agg1#85,agg2#86,agg3#87,agg4#88]), [i_item_id#19, s_state#17, g_state#84, agg1#85, agg2#86, agg3#87, agg4#88], 100, [i_item_id#19 ASC NULLS FIRST, s_state#17 ASC NULLS FIRST], [i_item_id#19, s_state#17, g_state#84, agg1#85, agg2#86, agg3#87, agg4#88] -(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] +(69) ColumnarToRow [codegen id : 1] +Input [7]: [i_item_id#19, s_state#17, g_state#84, agg1#85, agg2#86, agg3#87, agg4#88] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 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 (74) ++- * ColumnarToRow (73) + +- CometProject (72) + +- CometFilter (71) + +- CometScan parquet spark_catalog.default.date_dim (70) -(72) Scan parquet spark_catalog.default.date_dim +(70) 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 -(73) CometFilter +(71) 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)) -(74) CometProject +(72) CometProject Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(75) ColumnarToRow [codegen id : 1] +(73) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(76) BroadcastExchange +(74) BroadcastExchange Input [1]: [d_date_sk#14] 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#48 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 28 Hosting Expression = ss_sold_date_sk#39 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#85 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#66 IN dynamicpruning#9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/simplified.txt index de75d46ee..43f9cbc6e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/simplified.txt @@ -1,93 +1,81 @@ -TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - Union - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count,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,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] - CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_state] #5 - CometFilter [s_store_sk,s_state] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometFilter [i_item_sk,i_item_id] - 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] #7 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,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,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] - CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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 - ReusedExchange [cd_demo_sk] #3 - ReusedExchange [d_date_sk] #4 - CometBroadcastExchange [s_store_sk] #8 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - 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 #9 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] - CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk] - CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] - CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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 - ReusedExchange [cd_demo_sk] #3 - ReusedExchange [d_date_sk] #4 - ReusedExchange [s_store_sk] #8 - CometBroadcastExchange [i_item_sk] #10 - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] + CometUnion [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] + CometHashAggregate [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] + CometColumnarExchange [i_item_id,s_state] #1 + CometHashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count,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,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id] + CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [cd_demo_sk] #3 + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [s_store_sk,s_state] #5 + CometFilter [s_store_sk,s_state] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometHashAggregate [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] + CometColumnarExchange [i_item_id] #7 + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,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,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] + CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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 + ReusedExchange [cd_demo_sk] #3 + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange [s_store_sk] #8 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + ReusedExchange [i_item_sk,i_item_id] #6 + CometHashAggregate [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] + CometColumnarExchange #9 + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] + CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [agg1,agg2,agg3,agg4] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk] + CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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 + ReusedExchange [cd_demo_sk] #3 + ReusedExchange [d_date_sk] #4 + ReusedExchange [s_store_sk] #8 + CometBroadcastExchange [i_item_sk] #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-spark3_5/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/explain.txt index 92e11aabe..d978de885 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/explain.txt @@ -1,37 +1,36 @@ == Physical Plan == -* 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) +* ColumnarToRow (32) ++- CometSort (31) + +- CometColumnarExchange (30) + +- CometProject (29) + +- CometBroadcastHashJoin (28) + :- CometFilter (24) + : +- CometHashAggregate (23) + : +- CometColumnarExchange (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) + +- CometBroadcastExchange (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.customer (25) (1) Scan parquet spark_catalog.default.store_sales @@ -135,90 +134,84 @@ 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)] -(22) ColumnarToRow [codegen id : 1] +(22) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(24) HashAggregate [codegen id : 3] +(23) CometHashAggregate 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)#17] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#17 AS cnt#18] -(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)) +(24) CometFilter +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] +Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) -(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] +(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] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(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) +(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) -(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] +(27) CometBroadcastExchange +Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Arguments: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -(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] +(28) CometBroadcastHashJoin +Left output [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] +Right output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight -(30) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#19] -Join type: Inner -Join condition: None +(29) CometProject +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] +Arguments: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17], [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -(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] +(30) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] -(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] +(31) CometSort +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, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17], [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] -(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 +(32) ColumnarToRow [codegen id : 1] +Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (38) -+- * ColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.date_dim (34) +BroadcastExchange (37) ++- * ColumnarToRow (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) -(34) Scan parquet spark_catalog.default.date_dim +(33) 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 -(35) CometFilter +(34) 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)) -(36) CometProject +(35) CometProject Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(37) ColumnarToRow [codegen id : 1] +(36) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(38) BroadcastExchange +(37) BroadcastExchange Input [1]: [d_date_sk#7] -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-spark3_5/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt index 814e543ed..fafc612d6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt @@ -1,50 +1,42 @@ -WholeStageCodegen (4) - Sort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,count] - CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] - CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] - CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #5 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange [hd_demo_sk] #6 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 + CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometFilter [ss_ticket_number,ss_customer_sk,cnt] + CometHashAggregate [ss_ticket_number,ss_customer_sk,cnt,count,count(1)] + CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] + CometProject [ss_customer_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange [s_store_sk] #5 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_county] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county] + CometBroadcastExchange [hd_demo_sk] #6 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] #7 + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/explain.txt index 790000085..721036bfb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/explain.txt @@ -1,48 +1,50 @@ == Physical Plan == -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) +TakeOrderedAndProject (46) ++- * HashAggregate (45) + +- * ColumnarToRow (44) + +- CometColumnarExchange (43) + +- RowToColumnar (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 @@ -238,50 +240,56 @@ Functions [10]: [partial_count(1), partial_avg(cd_dep_count#25), partial_max(cd_ 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] -(42) Exchange +(42) RowToColumnar 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] -(43) HashAggregate [codegen id : 6] +(43) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=5] + +(44) ColumnarToRow [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] + +(45) 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] -(44) TakeOrderedAndProject +(46) 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 = 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) +BroadcastExchange (51) ++- * ColumnarToRow (50) + +- CometProject (49) + +- CometFilter (48) + +- CometScan parquet spark_catalog.default.date_dim (47) -(45) Scan parquet spark_catalog.default.date_dim +(47) 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 -(46) CometFilter +(48) 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)) -(47) CometProject +(49) CometProject Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(48) ColumnarToRow [codegen id : 1] +(50) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(49) BroadcastExchange +(51) BroadcastExchange Input [1]: [d_date_sk#9] 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-spark3_5/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/simplified.txt index 0b55e23ab..221648657 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/simplified.txt @@ -1,70 +1,72 @@ 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 (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 (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] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] - CometFilter [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] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,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 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + RowToColumnar + 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] + Project [c_current_cdemo_sk,ca_state] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] + Filter [exists,exists] + BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,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 + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [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] + CometBroadcastExchange [ss_customer_sk] #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,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 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [cs_ship_customer_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,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 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) + BroadcastExchange #8 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - 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] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + 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-spark3_5/q35a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/explain.txt index 215d4240e..476dab310 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (38) -+- * HashAggregate (37) - +- Exchange (36) - +- * ColumnarToRow (35) +* ColumnarToRow (38) ++- CometTakeOrderedAndProject (37) + +- CometHashAggregate (36) + +- CometColumnarExchange (35) +- CometHashAggregate (34) +- CometProject (33) +- CometBroadcastHashJoin (32) @@ -202,23 +202,21 @@ Input [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd 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] +(35) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(36) CometHashAggregate 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] +(37) CometTakeOrderedAndProject +Input [18]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#41, avg(cd_dep_count)#42, max(cd_dep_count)#43, sum(cd_dep_count)#44, cd_dep_employed_count#26, cnt2#45, avg(cd_dep_employed_count)#46, max(cd_dep_employed_count)#47, sum(cd_dep_employed_count)#48, cd_dep_college_count#27, cnt3#49, avg(cd_dep_college_count)#50, max(cd_dep_college_count)#51, sum(cd_dep_college_count)#52] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[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], output=[ca_state#21,cd_gender#23,cd_marital_status#24,cd_dep_count#25,cnt1#41,avg(cd_dep_count)#42,max(cd_dep_count)#43,sum(cd_dep_count)#44,cd_dep_employed_count#26,cnt2#45,avg(cd_dep_employed_count)#46,max(cd_dep_employed_count)#47,sum(cd_dep_employed_count)#48,cd_dep_college_count#27,cnt3#49,avg(cd_dep_college_count)#50,max(cd_dep_college_count)#51,sum(cd_dep_college_count)#52]), [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#41, avg(cd_dep_count)#42, max(cd_dep_count)#43, sum(cd_dep_count)#44, cd_dep_employed_count#26, cnt2#45, avg(cd_dep_employed_count)#46, max(cd_dep_employed_count)#47, sum(cd_dep_employed_count)#48, cd_dep_college_count#27, cnt3#49, avg(cd_dep_college_count)#50, max(cd_dep_college_count)#51, sum(cd_dep_college_count)#52], 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#41, avg(cd_dep_count)#42, max(cd_dep_count)#43, sum(cd_dep_count)#44, cd_dep_employed_count#26, cnt2#45, avg(cd_dep_employed_count)#46, max(cd_dep_employed_count)#47, sum(cd_dep_employed_count)#48, cd_dep_college_count#27, cnt3#49, avg(cd_dep_college_count)#50, max(cd_dep_college_count)#51, sum(cd_dep_college_count)#52] + +(38) ColumnarToRow [codegen id : 1] +Input [18]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#41, avg(cd_dep_count)#42, max(cd_dep_count)#43, sum(cd_dep_count)#44, cd_dep_employed_count#26, cnt2#45, avg(cd_dep_employed_count)#46, max(cd_dep_employed_count)#47, sum(cd_dep_employed_count)#48, cd_dep_college_count#27, cnt3#49, avg(cd_dep_college_count)#50, max(cd_dep_college_count)#51, sum(cd_dep_college_count)#52] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/simplified.txt index 8d73022ee..25f72e37b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/simplified.txt @@ -1,52 +1,50 @@ -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 (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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [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] - CometProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometBroadcastHashJoin [c_current_cdemo_sk,ca_state,cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometProject [c_current_cdemo_sk,ca_state] - CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk,ca_state] - CometProject [c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customsk] - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] - CometFilter [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] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [customsk] #5 - CometUnion [customsk] - CometProject [ws_bill_customer_sk] [customsk] - CometBroadcastHashJoin [ws_bill_customer_sk,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_ship_customer_sk,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 [ca_address_sk,ca_state] #6 - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - 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] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cd_dep_employed_count,cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cd_dep_college_count,cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] + CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cd_dep_employed_count,cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cd_dep_college_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,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)] + CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + CometHashAggregate [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] + CometProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometBroadcastHashJoin [c_current_cdemo_sk,ca_state,cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometProject [c_current_cdemo_sk,ca_state] + CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk,ca_state] + CometProject [c_current_cdemo_sk,c_current_addr_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customsk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [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] + CometBroadcastExchange [ss_customer_sk] #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [customsk] #5 + CometUnion [customsk] + CometProject [ws_bill_customer_sk] [customsk] + CometBroadcastHashJoin [ws_bill_customer_sk,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_ship_customer_sk,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 [ca_address_sk,ca_state] #6 + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + 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-spark3_5/q36a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/explain.txt index 51c37ba94..942fd184c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/explain.txt @@ -2,45 +2,45 @@ 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) + +- * ColumnarToRow (39) + +- CometSort (38) + +- CometColumnarExchange (37) + +- CometHashAggregate (36) + +- CometColumnarExchange (35) + +- CometHashAggregate (34) + +- CometUnion (33) + :- CometHashAggregate (22) + : +- CometColumnarExchange (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) + :- CometHashAggregate (27) + : +- CometColumnarExchange (26) + : +- CometHashAggregate (25) + : +- CometHashAggregate (24) + : +- ReusedExchange (23) + +- CometHashAggregate (32) + +- CometColumnarExchange (31) + +- CometHashAggregate (30) + +- CometHashAggregate (29) + +- ReusedExchange (28) (1) Scan parquet spark_catalog.default.store_sales @@ -140,115 +140,100 @@ 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))] -(21) ColumnarToRow [codegen id : 1] +(21) CometColumnarExchange Input [4]: [i_category#11, i_class#10, sum#14, sum#15] +Arguments: hashpartitioning(i_category#11, i_class#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(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] - -(23) HashAggregate [codegen id : 2] +(22) CometHashAggregate 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))#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] - -(24) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#22, i_class#23, sum#24, sum#25] - -(25) HashAggregate [codegen id : 4] -Input [4]: [i_category#22, i_class#23, sum#24, sum#25] -Keys [2]: [i_category#22, i_class#23] -Functions [2]: [sum(UnscaledValue(ss_net_profit#26)), sum(UnscaledValue(ss_ext_sales_price#27))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#26))#28, sum(UnscaledValue(ss_ext_sales_price#27))#29] -Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#26))#28,17,2) AS ss_net_profit#30, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#27))#29,17,2) AS ss_ext_sales_price#31, i_category#22] - -(26) HashAggregate [codegen id : 4] -Input [3]: [ss_net_profit#30, ss_ext_sales_price#31, i_category#22] -Keys [1]: [i_category#22] -Functions [2]: [partial_sum(ss_net_profit#30), partial_sum(ss_ext_sales_price#31)] -Aggregate Attributes [4]: [sum#32, isEmpty#33, sum#34, isEmpty#35] -Results [5]: [i_category#22, sum#36, isEmpty#37, sum#38, isEmpty#39] - -(27) Exchange -Input [5]: [i_category#22, sum#36, isEmpty#37, sum#38, isEmpty#39] -Arguments: hashpartitioning(i_category#22, 5), ENSURE_REQUIREMENTS, [plan_id=2] - -(28) HashAggregate [codegen id : 5] -Input [5]: [i_category#22, sum#36, isEmpty#37, sum#38, isEmpty#39] -Keys [1]: [i_category#22] -Functions [2]: [sum(ss_net_profit#30), sum(ss_ext_sales_price#31)] -Aggregate Attributes [2]: [sum(ss_net_profit#30)#40, sum(ss_ext_sales_price#31)#41] -Results [6]: [cast((sum(ss_net_profit#30)#40 / sum(ss_ext_sales_price#31)#41) as decimal(38,20)) AS gross_margin#42, i_category#22, null AS i_class#43, 0 AS t_category#44, 1 AS t_class#45, 1 AS lochierarchy#46] - -(29) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#47, i_class#48, sum#49, sum#50] - -(30) HashAggregate [codegen id : 7] -Input [4]: [i_category#47, i_class#48, sum#49, sum#50] -Keys [2]: [i_category#47, i_class#48] -Functions [2]: [sum(UnscaledValue(ss_net_profit#51)), sum(UnscaledValue(ss_ext_sales_price#52))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#51))#28, sum(UnscaledValue(ss_ext_sales_price#52))#29] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#51))#28,17,2) AS ss_net_profit#53, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#52))#29,17,2) AS ss_ext_sales_price#54] - -(31) HashAggregate [codegen id : 7] -Input [2]: [ss_net_profit#53, ss_ext_sales_price#54] + +(23) ReusedExchange [Reuses operator id: 21] +Output [4]: [i_category#16, i_class#17, sum#18, sum#19] + +(24) CometHashAggregate +Input [4]: [i_category#16, i_class#17, sum#18, sum#19] +Keys [2]: [i_category#16, i_class#17] +Functions [2]: [sum(UnscaledValue(ss_net_profit#20)), sum(UnscaledValue(ss_ext_sales_price#21))] + +(25) CometHashAggregate +Input [3]: [ss_net_profit#22, ss_ext_sales_price#23, i_category#16] +Keys [1]: [i_category#16] +Functions [2]: [partial_sum(ss_net_profit#22), partial_sum(ss_ext_sales_price#23)] + +(26) CometColumnarExchange +Input [5]: [i_category#16, sum#24, isEmpty#25, sum#26, isEmpty#27] +Arguments: hashpartitioning(i_category#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(27) CometHashAggregate +Input [5]: [i_category#16, sum#24, isEmpty#25, sum#26, isEmpty#27] +Keys [1]: [i_category#16] +Functions [2]: [sum(ss_net_profit#22), sum(ss_ext_sales_price#23)] + +(28) ReusedExchange [Reuses operator id: 21] +Output [4]: [i_category#28, i_class#29, sum#30, sum#31] + +(29) CometHashAggregate +Input [4]: [i_category#28, i_class#29, sum#30, sum#31] +Keys [2]: [i_category#28, i_class#29] +Functions [2]: [sum(UnscaledValue(ss_net_profit#32)), sum(UnscaledValue(ss_ext_sales_price#33))] + +(30) CometHashAggregate +Input [2]: [ss_net_profit#34, ss_ext_sales_price#35] Keys: [] -Functions [2]: [partial_sum(ss_net_profit#53), partial_sum(ss_ext_sales_price#54)] -Aggregate Attributes [4]: [sum#55, isEmpty#56, sum#57, isEmpty#58] -Results [4]: [sum#59, isEmpty#60, sum#61, isEmpty#62] +Functions [2]: [partial_sum(ss_net_profit#34), partial_sum(ss_ext_sales_price#35)] -(32) Exchange -Input [4]: [sum#59, isEmpty#60, sum#61, isEmpty#62] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] +(31) CometColumnarExchange +Input [4]: [sum#36, isEmpty#37, sum#38, isEmpty#39] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(33) HashAggregate [codegen id : 8] -Input [4]: [sum#59, isEmpty#60, sum#61, isEmpty#62] +(32) CometHashAggregate +Input [4]: [sum#36, isEmpty#37, sum#38, isEmpty#39] Keys: [] -Functions [2]: [sum(ss_net_profit#53), sum(ss_ext_sales_price#54)] -Aggregate Attributes [2]: [sum(ss_net_profit#53)#63, sum(ss_ext_sales_price#54)#64] -Results [6]: [cast((sum(ss_net_profit#53)#63 / sum(ss_ext_sales_price#54)#64) as decimal(38,20)) AS gross_margin#65, null AS i_category#66, null AS i_class#67, 1 AS t_category#68, 1 AS t_class#69, 2 AS lochierarchy#70] +Functions [2]: [sum(ss_net_profit#34), sum(ss_ext_sales_price#35)] -(34) Union +(33) CometUnion +Child 0 Input [6]: [gross_margin#40, i_category#11, i_class#10, t_category#41, t_class#42, lochierarchy#43] +Child 1 Input [6]: [gross_margin#44, i_category#16, i_class#45, t_category#46, t_class#47, lochierarchy#48] +Child 2 Input [6]: [gross_margin#49, i_category#50, i_class#51, t_category#52, t_class#53, lochierarchy#54] -(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] +(34) CometHashAggregate +Input [6]: [gross_margin#40, i_category#11, i_class#10, t_category#41, t_class#42, lochierarchy#43] +Keys [6]: [gross_margin#40, i_category#11, i_class#10, t_category#41, t_class#42, lochierarchy#43] Functions: [] -Aggregate Attributes: [] -Results [6]: [gross_margin#18, i_category#11, i_class#10, t_category#19, t_class#20, lochierarchy#21] -(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] +(35) CometColumnarExchange +Input [6]: [gross_margin#40, i_category#11, i_class#10, t_category#41, t_class#42, lochierarchy#43] +Arguments: hashpartitioning(gross_margin#40, i_category#11, i_class#10, t_category#41, t_class#42, lochierarchy#43, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(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] +(36) CometHashAggregate +Input [6]: [gross_margin#40, i_category#11, i_class#10, t_category#41, t_class#42, lochierarchy#43] +Keys [6]: [gross_margin#40, i_category#11, i_class#10, t_category#41, t_class#42, lochierarchy#43] Functions: [] -Aggregate Attributes: [] -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#71] -(38) Exchange -Input [5]: [gross_margin#18, i_category#11, i_class#10, lochierarchy#21, _w0#71] -Arguments: hashpartitioning(lochierarchy#21, _w0#71, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(37) CometColumnarExchange +Input [5]: [gross_margin#40, i_category#11, i_class#10, lochierarchy#43, _w0#55] +Arguments: hashpartitioning(lochierarchy#43, _w0#55, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(38) CometSort +Input [5]: [gross_margin#40, i_category#11, i_class#10, lochierarchy#43, _w0#55] +Arguments: [gross_margin#40, i_category#11, i_class#10, lochierarchy#43, _w0#55], [lochierarchy#43 ASC NULLS FIRST, _w0#55 ASC NULLS FIRST, gross_margin#40 ASC NULLS FIRST] -(39) Sort [codegen id : 11] -Input [5]: [gross_margin#18, i_category#11, i_class#10, lochierarchy#21, _w0#71] -Arguments: [lochierarchy#21 ASC NULLS FIRST, _w0#71 ASC NULLS FIRST, gross_margin#18 ASC NULLS FIRST], false, 0 +(39) ColumnarToRow [codegen id : 1] +Input [5]: [gross_margin#40, i_category#11, i_class#10, lochierarchy#43, _w0#55] (40) Window -Input [5]: [gross_margin#18, i_category#11, i_class#10, lochierarchy#21, _w0#71] -Arguments: [rank(gross_margin#18) windowspecdefinition(lochierarchy#21, _w0#71, gross_margin#18 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#72], [lochierarchy#21, _w0#71], [gross_margin#18 ASC NULLS FIRST] +Input [5]: [gross_margin#40, i_category#11, i_class#10, lochierarchy#43, _w0#55] +Arguments: [rank(gross_margin#40) windowspecdefinition(lochierarchy#43, _w0#55, gross_margin#40 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#56], [lochierarchy#43, _w0#55], [gross_margin#40 ASC NULLS FIRST] -(41) Project [codegen id : 12] -Output [5]: [gross_margin#18, i_category#11, i_class#10, lochierarchy#21, rank_within_parent#72] -Input [6]: [gross_margin#18, i_category#11, i_class#10, lochierarchy#21, _w0#71, rank_within_parent#72] +(41) Project [codegen id : 2] +Output [5]: [gross_margin#40, i_category#11, i_class#10, lochierarchy#43, rank_within_parent#56] +Input [6]: [gross_margin#40, i_category#11, i_class#10, lochierarchy#43, _w0#55, rank_within_parent#56] (42) TakeOrderedAndProject -Input [5]: [gross_margin#18, i_category#11, i_class#10, lochierarchy#21, rank_within_parent#72] -Arguments: 100, [lochierarchy#21 DESC NULLS LAST, CASE WHEN (lochierarchy#21 = 0) THEN i_category#11 END ASC NULLS FIRST, rank_within_parent#72 ASC NULLS FIRST], [gross_margin#18, i_category#11, i_class#10, lochierarchy#21, rank_within_parent#72] +Input [5]: [gross_margin#40, i_category#11, i_class#10, lochierarchy#43, rank_within_parent#56] +Arguments: 100, [lochierarchy#43 DESC NULLS LAST, CASE WHEN (lochierarchy#43 = 0) THEN i_category#11 END ASC NULLS FIRST, rank_within_parent#56 ASC NULLS FIRST], [gross_margin#40, i_category#11, i_class#10, lochierarchy#43, rank_within_parent#56] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/simplified.txt index 256fddfc8..9fc6f31f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/simplified.txt @@ -1,70 +1,54 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (12) + WholeStageCodegen (2) Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [gross_margin,lochierarchy,_w0] - WholeStageCodegen (11) - Sort [lochierarchy,_w0,gross_margin] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [lochierarchy,_w0] #1 - 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 (9) - HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - InputAdapter - Union - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_category,i_class,sum,sum,ss_net_profit,ss_ext_sales_price] - CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] - CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] - CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_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 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometFilter [i_item_sk,i_class,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #7 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - 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] #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 (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 #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 - ReusedExchange [i_category,i_class,sum,sum] #3 + CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] + CometColumnarExchange [lochierarchy,_w0] #1 + CometHashAggregate [gross_margin,i_category,i_class,lochierarchy,_w0,t_category,t_class] + CometColumnarExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 + CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] + CometUnion [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] + CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,sum,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange [i_category,i_class] #3 + CometHashAggregate [i_category,i_class,sum,sum,ss_net_profit,ss_ext_sales_price] + CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] + CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_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 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometFilter [i_item_sk,i_class,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometBroadcastExchange [s_store_sk] #7 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty,sum(ss_net_profit),sum(ss_ext_sales_price)] + CometColumnarExchange [i_category] #8 + CometHashAggregate [i_category,sum,isEmpty,sum,isEmpty,ss_net_profit,ss_ext_sales_price] + CometHashAggregate [ss_net_profit,ss_ext_sales_price,i_category,i_class,sum,sum,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] + ReusedExchange [i_category,i_class,sum,sum] #3 + CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty,sum(ss_net_profit),sum(ss_ext_sales_price)] + CometColumnarExchange #9 + CometHashAggregate [sum,isEmpty,sum,isEmpty,ss_net_profit,ss_ext_sales_price] + CometHashAggregate [ss_net_profit,ss_ext_sales_price,i_category,i_class,sum,sum,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] + ReusedExchange [i_category,i_class,sum,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/explain.txt index 41d045790..b6ab28625 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/explain.txt @@ -2,18 +2,18 @@ TakeOrderedAndProject (45) +- * Project (44) +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) + :- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) : :- * Project (28) : : +- * Filter (27) : : +- Window (26) : : +- * Filter (25) : : +- Window (24) - : : +- * Sort (23) - : : +- Exchange (22) - : : +- * HashAggregate (21) - : : +- Exchange (20) - : : +- * ColumnarToRow (19) + : : +- * ColumnarToRow (23) + : : +- CometSort (22) + : : +- CometColumnarExchange (21) + : : +- CometHashAggregate (20) + : : +- CometColumnarExchange (19) : : +- CometHashAggregate (18) : : +- CometProject (17) : : +- CometBroadcastHashJoin (16) @@ -32,18 +32,18 @@ TakeOrderedAndProject (45) : : +- CometBroadcastExchange (15) : : +- CometFilter (14) : : +- CometScan parquet spark_catalog.default.store (13) - : +- BroadcastExchange (35) - : +- * Project (34) - : +- Window (33) - : +- * Sort (32) - : +- Exchange (31) - : +- * HashAggregate (30) - : +- ReusedExchange (29) + : +- BroadcastExchange (34) + : +- * Project (33) + : +- Window (32) + : +- * ColumnarToRow (31) + : +- CometSort (30) + : +- ReusedExchange (29) +- BroadcastExchange (42) +- * Project (41) +- Window (40) - +- * Sort (39) - +- ReusedExchange (38) + +- * ColumnarToRow (39) + +- CometSort (38) + +- ReusedExchange (37) (1) Scan parquet spark_catalog.default.item @@ -135,120 +135,113 @@ Input [7]: [i_brand#2, i_category#3, ss_sales_price#6, d_year#10, d_moy#11, s_st 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))] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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#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 : 2] +(20) CometHashAggregate 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))#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#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] +(21) CometColumnarExchange +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, _w0#17] +Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(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 +(22) CometSort +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, _w0#17] +Arguments: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, _w0#17], [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] + +(23) ColumnarToRow [codegen id : 1] +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, _w0#17] (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#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] +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 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, 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#18], [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 : 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] +(25) Filter [codegen id : 2] +Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 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 [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 : 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 : 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#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum#27] - -(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#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 : 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#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 : 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#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 : 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)] +Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, _w0#17, rn#18] +Arguments: [avg(_w0#17) 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#19], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10] + +(27) Filter [codegen id : 7] +Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 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 : 7] +Output [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18] +Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, _w0#17, rn#18, avg_monthly_sales#19] + +(29) ReusedExchange [Reuses operator id: 21] +Output [7]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#16] + +(30) CometSort +Input [7]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#16] +Arguments: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#16], [i_category#20 ASC NULLS FIRST, i_brand#21 ASC NULLS FIRST, s_store_name#22 ASC NULLS FIRST, s_company_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] + +(31) ColumnarToRow [codegen id : 3] +Input [7]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#16] + +(32) Window +Input [7]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#16] +Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#20, i_brand#21, s_store_name#22, s_company_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#26], [i_category#20, i_brand#21, s_store_name#22, s_company_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] + +(33) Project [codegen id : 4] +Output [6]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, sum_sales#16 AS sum_sales#27, rn#26] +Input [8]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#16, rn#26] + +(34) BroadcastExchange +Input [6]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, sum_sales#27, rn#26] +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=3] + +(35) BroadcastHashJoin [codegen id : 7] +Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#18] +Right keys [5]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, (rn#26 + 1)] Join type: Inner Join condition: None -(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] +(36) Project [codegen id : 7] +Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, sum_sales#27] +Input [15]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, i_category#20, i_brand#21, s_store_name#22, s_company_name#23, sum_sales#27, rn#26] + +(37) ReusedExchange [Reuses operator id: 21] +Output [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#16] -(38) ReusedExchange [Reuses operator id: 31] -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] +(38) CometSort +Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#16] +Arguments: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#16], [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_name#30 ASC NULLS FIRST, s_company_name#31 ASC NULLS FIRST, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] -(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 +(39) ColumnarToRow [codegen id : 5] +Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#16] (40) Window -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] +Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#16] +Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#34], [i_category#28, i_brand#29, s_store_name#30, s_company_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] -(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] +(41) Project [codegen id : 6] +Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#16 AS sum_sales#35, rn#34] +Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#16, rn#34] (42) BroadcastExchange -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] +Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#35, rn#34] +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] -(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)] +(43) BroadcastHashJoin [codegen id : 7] +Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#18] +Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#34 - 1)] Join type: Inner Join condition: None -(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] +(44) Project [codegen id : 7] +Output [7]: [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#19, sum_sales#16, sum_sales#27 AS psum#36, sum_sales#35 AS nsum#37] +Input [16]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, sum_sales#27, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#35, rn#34] (45) TakeOrderedAndProject -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] +Input [7]: [i_category#3, 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_moy#11 ASC NULLS FIRST], [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#19, sum_sales#16, psum#36, nsum#37] ===== Subqueries ===== @@ -275,6 +268,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=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/simplified.txt index eeeb5ba36..3c44ef74c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] - WholeStageCodegen (13) + WholeStageCodegen (7) 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,67 +8,61 @@ 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 (4) + WholeStageCodegen (2) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (3) - Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum,ss_sales_price] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometFilter [i_item_sk,i_brand,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #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] + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 + CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum,sum(UnscaledValue(ss_sales_price))] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum,ss_sales_price] + CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #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 #7 - WholeStageCodegen (8) + WholeStageCodegen (4) 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 (7) - Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + WholeStageCodegen (3) + ColumnarToRow InputAdapter - 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 + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #1 InputAdapter - BroadcastExchange #9 - WholeStageCodegen (12) + BroadcastExchange #8 + WholeStageCodegen (6) 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 (11) - Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + WholeStageCodegen (5) + ColumnarToRow InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/explain.txt index 07bd1d0ca..ed68082f6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/explain.txt @@ -1,81 +1,83 @@ == Physical Plan == -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) +* ColumnarToRow (79) ++- CometTakeOrderedAndProject (78) + +- CometHashAggregate (77) + +- CometColumnarExchange (76) + +- RowToColumnar (75) + +- * HashAggregate (74) + +- Union (73) + :- * Project (26) + : +- * Filter (25) + : +- Window (24) + : +- * Sort (23) + : +- Window (22) + : +- * ColumnarToRow (21) + : +- CometSort (20) + : +- CometColumnarExchange (19) + : +- CometHashAggregate (18) + : +- CometColumnarExchange (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) + : +- * ColumnarToRow (44) + : +- CometSort (43) + : +- CometColumnarExchange (42) + : +- CometHashAggregate (41) + : +- CometColumnarExchange (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) + +- * ColumnarToRow (67) + +- CometSort (66) + +- CometColumnarExchange (65) + +- CometHashAggregate (64) + +- CometColumnarExchange (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 @@ -155,312 +157,310 @@ Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, w 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))] -(17) ColumnarToRow [codegen id : 1] +(17) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(19) HashAggregate [codegen id : 2] +(18) CometHashAggregate 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))#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] -(20) Exchange -Input [3]: [item#26, return_ratio#27, currency_ratio#28] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] +(19) CometColumnarExchange +Input [3]: [item#22, return_ratio#23, currency_ratio#24] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(20) CometSort +Input [3]: [item#22, return_ratio#23, currency_ratio#24] +Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] -(21) Sort [codegen id : 3] -Input [3]: [item#26, return_ratio#27, currency_ratio#28] -Arguments: [return_ratio#27 ASC NULLS FIRST], false, 0 +(21) ColumnarToRow [codegen id : 1] +Input [3]: [item#22, return_ratio#23, currency_ratio#24] (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] +Input [3]: [item#22, return_ratio#23, currency_ratio#24] +Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] -(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 +(23) Sort [codegen id : 2] +Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] +Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 (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] +Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] +Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] -(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)) +(25) Filter [codegen id : 3] +Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] +Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) -(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] +(26) Project [codegen id : 3] +Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] (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] +Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#37), dynamicpruningexpression(cs_sold_date_sk#37 IN dynamicpruning#38)] +PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_sold_date_sk#33 IN dynamicpruning#34)] 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 (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)) +Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] +Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) (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] +Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] +Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] (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] +Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] +Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] (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] +Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] 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 (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)) +Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] +Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) (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] +Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] +Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] (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 +Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] +Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] +Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], 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] +Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] +Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] (36) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#44] +Output [1]: [d_date_sk#40] (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 +Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] +Right output [1]: [d_date_sk#40] +Arguments: [cs_sold_date_sk#33], [d_date_sk#40], 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] +Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] +Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] (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))] +Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] +Keys [1]: [cs_item_sk#28] +Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 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] +(40) CometColumnarExchange +Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] +Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(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] +(41) CometHashAggregate +Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] +Keys [1]: [cs_item_sk#28] +Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(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] +(42) CometColumnarExchange +Input [3]: [item#47, return_ratio#48, currency_ratio#49] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(43) Exchange -Input [3]: [item#55, return_ratio#56, currency_ratio#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +(43) CometSort +Input [3]: [item#47, return_ratio#48, currency_ratio#49] +Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] -(44) Sort [codegen id : 8] -Input [3]: [item#55, return_ratio#56, currency_ratio#57] -Arguments: [return_ratio#56 ASC NULLS FIRST], false, 0 +(44) ColumnarToRow [codegen id : 4] +Input [3]: [item#47, return_ratio#48, currency_ratio#49] (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] +Input [3]: [item#47, return_ratio#48, currency_ratio#49] +Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] -(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 +(46) Sort [codegen id : 5] +Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] +Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 (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] +Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] +Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] -(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)) +(48) Filter [codegen id : 6] +Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] +Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) -(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] +(49) Project [codegen id : 6] +Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] +Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] (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] +Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#66), dynamicpruningexpression(ss_sold_date_sk#66 IN dynamicpruning#67)] +PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] 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 (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)) +Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] +Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) (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] +Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] +Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] (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] +Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] +Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] (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] +Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] 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 (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)) +Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] +Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) (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] +Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] +Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] (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 +Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] +Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] +Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], 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] +Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] +Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] (59) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#73] +Output [1]: [d_date_sk#65] (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 +Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] +Right output [1]: [d_date_sk#65] +Arguments: [ss_sold_date_sk#58], [d_date_sk#65], 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] +Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] +Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] (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))] +Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] +Keys [1]: [ss_item_sk#53] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 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] +(63) CometColumnarExchange +Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] +Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(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] +(64) CometHashAggregate +Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] +Keys [1]: [ss_item_sk#53] +Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(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] +(65) CometColumnarExchange +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(66) Exchange -Input [3]: [item#84, return_ratio#85, currency_ratio#86] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +(66) CometSort +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] -(67) Sort [codegen id : 13] -Input [3]: [item#84, return_ratio#85, currency_ratio#86] -Arguments: [return_ratio#85 ASC NULLS FIRST], false, 0 +(67) ColumnarToRow [codegen id : 7] +Input [3]: [item#72, return_ratio#73, currency_ratio#74] (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] +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] -(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 +(69) Sort [codegen id : 8] +Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] +Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 (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] +Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] +Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] -(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)) +(71) Filter [codegen id : 9] +Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] +Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) -(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] +(72) Project [codegen id : 9] +Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] +Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] (73) Union -(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] +(74) HashAggregate [codegen id : 10] +Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(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] +(75) RowToColumnar +Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(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] +(76) CometColumnarExchange +Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(77) CometHashAggregate +Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] -(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] +(78) CometTakeOrderedAndProject +Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST,item#22 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST, item#22 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] + +(79) ColumnarToRow [codegen id : 11] +Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (82) -+- * ColumnarToRow (81) - +- CometProject (80) - +- CometFilter (79) - +- CometScan parquet spark_catalog.default.date_dim (78) +BroadcastExchange (84) ++- * ColumnarToRow (83) + +- CometProject (82) + +- CometFilter (81) + +- CometScan parquet spark_catalog.default.date_dim (80) -(78) Scan parquet spark_catalog.default.date_dim +(80) 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 -(79) CometFilter +(81) 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)) -(80) CometProject +(82) CometProject Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(81) ColumnarToRow [codegen id : 1] +(83) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(82) BroadcastExchange +(84) 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 = 27 Hosting Expression = cs_sold_date_sk#37 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#66 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/simplified.txt index 0e6b65b06..0d02a9c39 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/simplified.txt @@ -1,121 +1,111 @@ -TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] - WholeStageCodegen (17) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Exchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (16) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (5) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (4) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (3) - Sort [return_ratio] - InputAdapter - Exchange #2 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,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_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - 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 (9) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (8) - Sort [return_ratio] - InputAdapter - 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] #8 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometHashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,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_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_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] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - 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 (14) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (13) - Sort [return_ratio] - InputAdapter - 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] #11 - WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometHashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,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_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_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] - ReusedSubquery [d_date_sk] #1 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - ReusedExchange [d_date_sk] #6 +WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] + CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] + CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] + InputAdapter + Union + WholeStageCodegen (3) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (2) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #2 + CometHashAggregate [item,return_ratio,currency_ratio,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))] + CometColumnarExchange [ws_item_sk] #3 + CometHashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,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_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + WholeStageCodegen (6) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (5) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #7 + CometHashAggregate [item,return_ratio,currency_ratio,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))] + CometColumnarExchange [cs_item_sk] #8 + CometHashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,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_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_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] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (9) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (8) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #10 + CometHashAggregate [item,return_ratio,currency_ratio,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))] + CometColumnarExchange [ss_item_sk] #11 + CometHashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,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_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_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] + ReusedSubquery [d_date_sk] #1 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/explain.txt index 1bcaa1ea1..20a3f1c52 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/explain.txt @@ -1,74 +1,84 @@ == Physical Plan == -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) +TakeOrderedAndProject (80) ++- * Filter (79) + +- * HashAggregate (78) + +- * HashAggregate (77) + +- * Project (76) + +- * BroadcastHashJoin Inner BuildRight (75) + :- Window (68) + : +- * ColumnarToRow (67) + : +- CometSort (66) + : +- CometColumnarExchange (65) + : +- CometProject (64) + : +- CometFilter (63) + : +- CometSortMergeJoin (62) + : :- CometSort (32) + : : +- CometColumnarExchange (31) + : : +- RowToColumnar (30) + : : +- * HashAggregate (29) + : : +- * ColumnarToRow (28) + : : +- CometColumnarExchange (27) + : : +- RowToColumnar (26) + : : +- * HashAggregate (25) + : : +- * Project (24) + : : +- * BroadcastHashJoin Inner BuildRight (23) + : : :- * Project (16) + : : : +- Window (15) + : : : +- * ColumnarToRow (14) + : : : +- CometSort (13) + : : : +- CometColumnarExchange (12) + : : : +- CometHashAggregate (11) + : : : +- CometColumnarExchange (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 (22) + : : +- * Project (21) + : : +- Window (20) + : : +- * ColumnarToRow (19) + : : +- CometSort (18) + : : +- ReusedExchange (17) + : +- CometSort (61) + : +- CometColumnarExchange (60) + : +- RowToColumnar (59) + : +- * HashAggregate (58) + : +- * ColumnarToRow (57) + : +- CometColumnarExchange (56) + : +- RowToColumnar (55) + : +- * HashAggregate (54) + : +- * Project (53) + : +- * BroadcastHashJoin Inner BuildRight (52) + : :- * Project (45) + : : +- Window (44) + : : +- * ColumnarToRow (43) + : : +- CometSort (42) + : : +- CometColumnarExchange (41) + : : +- CometHashAggregate (40) + : : +- CometColumnarExchange (39) + : : +- CometHashAggregate (38) + : : +- CometProject (37) + : : +- CometBroadcastHashJoin (36) + : : :- CometFilter (34) + : : : +- CometScan parquet spark_catalog.default.store_sales (33) + : : +- ReusedExchange (35) + : +- BroadcastExchange (51) + : +- * Project (50) + : +- Window (49) + : +- * ColumnarToRow (48) + : +- CometSort (47) + : +- ReusedExchange (46) + +- BroadcastExchange (74) + +- * Project (73) + +- Window (72) + +- * ColumnarToRow (71) + +- CometSort (70) + +- ReusedExchange (69) (1) Scan parquet spark_catalog.default.web_sales @@ -116,314 +126,339 @@ 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))] -(10) ColumnarToRow [codegen id : 1] +(10) CometColumnarExchange Input [3]: [ws_item_sk#1, d_date#6, sum#8] +Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(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] - -(12) HashAggregate [codegen id : 2] +(11) CometHashAggregate 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] -(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] +(12) CometColumnarExchange +Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(13) CometSort +Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] +Arguments: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(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 +(14) ColumnarToRow [codegen id : 1] +Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] (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] +Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] +Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(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] +(16) Project [codegen id : 4] +Output [4]: [item_sk#9, d_date#6, sumws#10, rk#11] +Input [5]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1, rk#11] -(17) ReusedExchange [Reuses operator id: 13] -Output [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] +(17) ReusedExchange [Reuses operator id: 12] +Output [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] -(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 +(18) CometSort +Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] +Arguments: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13], [ws_item_sk#13 ASC NULLS FIRST, d_date#12 ASC NULLS FIRST] -(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] +(19) ColumnarToRow [codegen id : 2] +Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] -(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] +(20) Window +Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] +Arguments: [row_number() windowspecdefinition(ws_item_sk#13, d_date#12 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#14], [ws_item_sk#13], [d_date#12 ASC NULLS FIRST] -(21) BroadcastExchange -Input [3]: [item_sk#16, sumws#17, rk#15] +(21) Project [codegen id : 3] +Output [3]: [item_sk#9 AS item_sk#15, sumws#10 AS sumws#16, rk#14] +Input [5]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13, rk#14] + +(22) BroadcastExchange +Input [3]: [item_sk#15, sumws#16, rk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(22) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [item_sk#10] -Right keys [1]: [item_sk#16] +(23) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [item_sk#9] +Right keys [1]: [item_sk#15] Join type: Inner -Join condition: (rk#12 >= rk#15) - -(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] - -(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] - -(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] - -(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] - -(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] - -(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 - -(29) 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#11 >= rk#14) + +(24) Project [codegen id : 4] +Output [4]: [item_sk#9, d_date#6, sumws#10, sumws#16] +Input [7]: [item_sk#9, d_date#6, sumws#10, rk#11, item_sk#15, sumws#16, rk#14] + +(25) HashAggregate [codegen id : 4] +Input [4]: [item_sk#9, d_date#6, sumws#10, sumws#16] +Keys [3]: [item_sk#9, d_date#6, sumws#10] +Functions [1]: [partial_sum(sumws#16)] +Aggregate Attributes [2]: [sum#17, isEmpty#18] +Results [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] + +(26) RowToColumnar +Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] + +(27) CometColumnarExchange +Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] +Arguments: hashpartitioning(item_sk#9, d_date#6, sumws#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(28) ColumnarToRow [codegen id : 5] +Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] + +(29) HashAggregate [codegen id : 5] +Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] +Keys [3]: [item_sk#9, d_date#6, sumws#10] +Functions [1]: [sum(sumws#16)] +Aggregate Attributes [1]: [sum(sumws#16)#21] +Results [3]: [item_sk#9, d_date#6, sum(sumws#16)#21 AS cume_sales#22] + +(30) RowToColumnar +Input [3]: [item_sk#9, d_date#6, cume_sales#22] + +(31) CometColumnarExchange +Input [3]: [item_sk#9, d_date#6, cume_sales#22] +Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(32) CometSort +Input [3]: [item_sk#9, d_date#6, cume_sales#22] +Arguments: [item_sk#9, d_date#6, cume_sales#22], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] + +(33) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] 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#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(30) CometFilter -Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_item_sk#24) +(34) CometFilter +Input [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] +Condition : isnotnull(ss_item_sk#23) + +(35) ReusedExchange [Reuses operator id: 6] +Output [2]: [d_date_sk#27, d_date#28] -(31) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#28, d_date#29] +(36) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] +Right output [2]: [d_date_sk#27, d_date#28] +Arguments: [ss_sold_date_sk#25], [d_date_sk#27], Inner, BuildRight -(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 +(37) CometProject +Input [5]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25, d_date_sk#27, d_date#28] +Arguments: [ss_item_sk#23, ss_sales_price#24, d_date#28], [ss_item_sk#23, ss_sales_price#24, d_date#28] -(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] +(38) CometHashAggregate +Input [3]: [ss_item_sk#23, ss_sales_price#24, d_date#28] +Keys [2]: [ss_item_sk#23, d_date#28] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#24))] -(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))] +(39) CometColumnarExchange +Input [3]: [ss_item_sk#23, d_date#28, sum#29] +Arguments: hashpartitioning(ss_item_sk#23, d_date#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(35) ColumnarToRow [codegen id : 11] -Input [3]: [ss_item_sk#24, d_date#29, sum#30] +(40) CometHashAggregate +Input [3]: [ss_item_sk#23, d_date#28, sum#29] +Keys [2]: [ss_item_sk#23, d_date#28] +Functions [1]: [sum(UnscaledValue(ss_sales_price#24))] -(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] +(41) CometColumnarExchange +Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] +Arguments: hashpartitioning(ss_item_sk#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(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))#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] +(42) CometSort +Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] +Arguments: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23], [ss_item_sk#23 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] -(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] +(43) ColumnarToRow [codegen id : 6] +Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -(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 +(44) Window +Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] +Arguments: [row_number() windowspecdefinition(ss_item_sk#23, d_date#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#32], [ss_item_sk#23], [d_date#28 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] +(45) Project [codegen id : 9] +Output [4]: [item_sk#30, d_date#28, sumss#31, rk#32] +Input [5]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23, rk#32] -(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] +(46) ReusedExchange [Reuses operator id: 41] +Output [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] -(42) ReusedExchange [Reuses operator id: 38] -Output [4]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36] +(47) CometSort +Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] +Arguments: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34], [ss_item_sk#34 ASC NULLS FIRST, d_date#33 ASC NULLS FIRST] -(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 +(48) ColumnarToRow [codegen id : 7] +Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] -(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] +(49) Window +Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] +Arguments: [row_number() windowspecdefinition(ss_item_sk#34, d_date#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#35], [ss_item_sk#34], [d_date#33 ASC NULLS FIRST] -(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] +(50) Project [codegen id : 8] +Output [3]: [item_sk#30 AS item_sk#36, sumss#31 AS sumss#37, rk#35] +Input [5]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34, rk#35] -(46) BroadcastExchange -Input [3]: [item_sk#38, sumss#39, rk#37] +(51) BroadcastExchange +Input [3]: [item_sk#36, sumss#37, rk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -(47) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_sk#32] -Right keys [1]: [item_sk#38] +(52) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [item_sk#30] +Right keys [1]: [item_sk#36] Join type: Inner -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#32, d_date#29] -Join type: FullOuter -Join condition: None - -(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) - -(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] - -(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] - -(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 - -(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] - -(60) ReusedExchange [Reuses operator id: 57] -Output [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] - -(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 - -(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] - -(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] - -(64) BroadcastExchange -Input [4]: [item_sk#52, web_sales#53, store_sales#54, rk#51] +Join condition: (rk#32 >= rk#35) + +(53) Project [codegen id : 9] +Output [4]: [item_sk#30, d_date#28, sumss#31, sumss#37] +Input [7]: [item_sk#30, d_date#28, sumss#31, rk#32, item_sk#36, sumss#37, rk#35] + +(54) HashAggregate [codegen id : 9] +Input [4]: [item_sk#30, d_date#28, sumss#31, sumss#37] +Keys [3]: [item_sk#30, d_date#28, sumss#31] +Functions [1]: [partial_sum(sumss#37)] +Aggregate Attributes [2]: [sum#38, isEmpty#39] +Results [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] + +(55) RowToColumnar +Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] + +(56) CometColumnarExchange +Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] +Arguments: hashpartitioning(item_sk#30, d_date#28, sumss#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(57) ColumnarToRow [codegen id : 10] +Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] + +(58) HashAggregate [codegen id : 10] +Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] +Keys [3]: [item_sk#30, d_date#28, sumss#31] +Functions [1]: [sum(sumss#37)] +Aggregate Attributes [1]: [sum(sumss#37)#42] +Results [3]: [item_sk#30, d_date#28, sum(sumss#37)#42 AS cume_sales#43] + +(59) RowToColumnar +Input [3]: [item_sk#30, d_date#28, cume_sales#43] + +(60) CometColumnarExchange +Input [3]: [item_sk#30, d_date#28, cume_sales#43] +Arguments: hashpartitioning(item_sk#30, d_date#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(61) CometSort +Input [3]: [item_sk#30, d_date#28, cume_sales#43] +Arguments: [item_sk#30, d_date#28, cume_sales#43], [item_sk#30 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] + +(62) CometSortMergeJoin +Left output [3]: [item_sk#9, d_date#6, cume_sales#22] +Right output [3]: [item_sk#30, d_date#28, cume_sales#43] +Arguments: [item_sk#9, d_date#6], [item_sk#30, d_date#28], FullOuter + +(63) CometFilter +Input [6]: [item_sk#9, d_date#6, cume_sales#22, item_sk#30, d_date#28, cume_sales#43] +Condition : isnotnull(CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END) + +(64) CometProject +Input [6]: [item_sk#9, d_date#6, cume_sales#22, item_sk#30, d_date#28, cume_sales#43] +Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END AS item_sk#44, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#28 END AS d_date#45, cume_sales#22 AS web_sales#46, cume_sales#43 AS store_sales#47] + +(65) CometColumnarExchange +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Arguments: hashpartitioning(item_sk#44, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] + +(66) CometSort +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST] + +(67) ColumnarToRow [codegen id : 11] +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] + +(68) Window +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Arguments: [row_number() windowspecdefinition(item_sk#44, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#48], [item_sk#44], [d_date#45 ASC NULLS FIRST] + +(69) ReusedExchange [Reuses operator id: 65] +Output [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] + +(70) CometSort +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST] + +(71) ColumnarToRow [codegen id : 22] +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] + +(72) Window +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Arguments: [row_number() windowspecdefinition(item_sk#44, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#49], [item_sk#44], [d_date#45 ASC NULLS FIRST] + +(73) Project [codegen id : 23] +Output [4]: [item_sk#44 AS item_sk#50, web_sales#46 AS web_sales#51, store_sales#47 AS store_sales#52, rk#49] +Input [5]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, rk#49] + +(74) BroadcastExchange +Input [4]: [item_sk#50, web_sales#51, store_sales#52, rk#49] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -(65) BroadcastHashJoin [codegen id : 46] -Left keys [1]: [item_sk#46] -Right keys [1]: [item_sk#52] +(75) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [item_sk#44] +Right keys [1]: [item_sk#50] Join type: Inner -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] +Join condition: (rk#48 >= rk#49) + +(76) Project [codegen id : 24] +Output [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_sales#51, store_sales#52] +Input [9]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, rk#48, item_sk#50, web_sales#51, store_sales#52, rk#49] + +(77) HashAggregate [codegen id : 24] +Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_sales#51, store_sales#52] +Keys [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Functions [2]: [partial_max(web_sales#51), partial_max(store_sales#52)] +Aggregate Attributes [2]: [max#53, max#54] +Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#55, max#56] + +(78) HashAggregate [codegen id : 24] +Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#55, max#56] +Keys [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Functions [2]: [max(web_sales#51), max(store_sales#52)] +Aggregate Attributes [2]: [max(web_sales#51)#57, max(store_sales#52)#58] +Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max(web_sales#51)#57 AS web_cumulative#59, max(store_sales#52)#58 AS store_cumulative#60] + +(79) Filter [codegen id : 24] +Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] +Condition : ((isnotnull(web_cumulative#59) AND isnotnull(store_cumulative#60)) AND (web_cumulative#59 > store_cumulative#60)) + +(80) TakeOrderedAndProject +Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] +Arguments: 100, [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (75) -+- * ColumnarToRow (74) - +- CometProject (73) - +- CometFilter (72) - +- CometScan parquet spark_catalog.default.date_dim (71) +BroadcastExchange (85) ++- * ColumnarToRow (84) + +- CometProject (83) + +- CometFilter (82) + +- CometScan parquet spark_catalog.default.date_dim (81) -(71) Scan parquet spark_catalog.default.date_dim +(81) 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 -(72) CometFilter +(82) 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)) -(73) CometProject +(83) 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] -(74) ColumnarToRow [codegen id : 1] +(84) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(75) BroadcastExchange +(85) 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 = 29 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/simplified.txt index e9c4d46ca..7fe01fcc8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (46) + WholeStageCodegen (24) 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,119 +7,114 @@ 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 (22) - Sort [item_sk,d_date] + WholeStageCodegen (11) + ColumnarToRow InputAdapter - Exchange [item_sk] #1 - 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 (10) - Sort [item_sk,d_date] - InputAdapter - Exchange [item_sk,d_date] #2 - 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 (8) + CometSort [item_sk,d_date,web_sales,store_sales] + CometColumnarExchange [item_sk] #1 + CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] + CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #2 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [item_sk,d_date,sumws] #3 + RowToColumnar + WholeStageCodegen (4) 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 (3) - Sort [ws_item_sk,d_date] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [ws_item_sk] #4 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ws_item_sk,d_date,sum,ws_sales_price] - CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometSort [item_sk,d_date,sumws,ws_item_sk] + CometColumnarExchange [ws_item_sk] #4 + CometHashAggregate [item_sk,d_date,sumws,ws_item_sk,sum,sum(UnscaledValue(ws_sales_price))] + CometColumnarExchange [ws_item_sk,d_date] #5 + CometHashAggregate [ws_item_sk,d_date,sum,ws_sales_price] + CometProject [ws_item_sk,ws_sales_price,d_date] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange [d_date_sk,d_date] #7 + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter BroadcastExchange #8 - WholeStageCodegen (7) + WholeStageCodegen (3) Project [item_sk,sumws,rk] InputAdapter Window [ws_item_sk,d_date] - WholeStageCodegen (6) - Sort [ws_item_sk,d_date] + WholeStageCodegen (2) + ColumnarToRow InputAdapter - ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 - InputAdapter - WholeStageCodegen (20) - Sort [item_sk,d_date] - InputAdapter - 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] #10 - WholeStageCodegen (18) + CometSort [item_sk,d_date,sumws,ws_item_sk] + ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #9 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [item_sk,d_date,sumss] #10 + RowToColumnar + WholeStageCodegen (9) 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 (13) - Sort [ss_item_sk,d_date] + WholeStageCodegen (6) + ColumnarToRow InputAdapter - 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] #12 - WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometHashAggregate [ss_item_sk,d_date,sum,ss_sales_price] - CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_date] #7 + CometSort [item_sk,d_date,sumss,ss_item_sk] + CometColumnarExchange [ss_item_sk] #11 + CometHashAggregate [item_sk,d_date,sumss,ss_item_sk,sum,sum(UnscaledValue(ss_sales_price))] + CometColumnarExchange [ss_item_sk,d_date] #12 + CometHashAggregate [ss_item_sk,d_date,sum,ss_sales_price] + CometProject [ss_item_sk,ss_sales_price,d_date] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_date] #7 InputAdapter BroadcastExchange #13 - WholeStageCodegen (17) + WholeStageCodegen (8) Project [item_sk,sumss,rk] InputAdapter Window [ss_item_sk,d_date] - WholeStageCodegen (16) - Sort [ss_item_sk,d_date] + WholeStageCodegen (7) + ColumnarToRow InputAdapter - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 + CometSort [item_sk,d_date,sumss,ss_item_sk] + ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 InputAdapter BroadcastExchange #14 - WholeStageCodegen (45) + WholeStageCodegen (23) Project [item_sk,web_sales,store_sales,rk] InputAdapter Window [item_sk,d_date] - WholeStageCodegen (44) - Sort [item_sk,d_date] + WholeStageCodegen (22) + ColumnarToRow InputAdapter - ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 + CometSort [item_sk,d_date,web_sales,store_sales] + ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/explain.txt index 1b66eb4da..59b4666bf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/explain.txt @@ -2,18 +2,18 @@ TakeOrderedAndProject (45) +- * Project (44) +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) + :- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) : :- * Project (28) : : +- * Filter (27) : : +- Window (26) : : +- * Filter (25) : : +- Window (24) - : : +- * Sort (23) - : : +- Exchange (22) - : : +- * HashAggregate (21) - : : +- Exchange (20) - : : +- * ColumnarToRow (19) + : : +- * ColumnarToRow (23) + : : +- CometSort (22) + : : +- CometColumnarExchange (21) + : : +- CometHashAggregate (20) + : : +- CometColumnarExchange (19) : : +- CometHashAggregate (18) : : +- CometProject (17) : : +- CometBroadcastHashJoin (16) @@ -32,18 +32,18 @@ TakeOrderedAndProject (45) : : +- CometBroadcastExchange (15) : : +- CometFilter (14) : : +- CometScan parquet spark_catalog.default.call_center (13) - : +- BroadcastExchange (35) - : +- * Project (34) - : +- Window (33) - : +- * Sort (32) - : +- Exchange (31) - : +- * HashAggregate (30) - : +- ReusedExchange (29) + : +- BroadcastExchange (34) + : +- * Project (33) + : +- Window (32) + : +- * ColumnarToRow (31) + : +- CometSort (30) + : +- ReusedExchange (29) +- BroadcastExchange (42) +- * Project (41) +- Window (40) - +- * Sort (39) - +- ReusedExchange (38) + +- * ColumnarToRow (39) + +- CometSort (38) + +- ReusedExchange (37) (1) Scan parquet spark_catalog.default.item @@ -135,120 +135,113 @@ Input [6]: [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_n 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))] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(20) Exchange -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 : 2] +(20) CometHashAggregate 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))#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#16, _w0#17] -Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(21) CometColumnarExchange +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16] +Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(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 +(22) CometSort +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16] +Arguments: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16], [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] + +(23) ColumnarToRow [codegen id : 1] +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16] (24) Window -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] +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16] +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#17], [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 : 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] +(25) Filter [codegen id : 2] +Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16, rn#17] 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#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 : 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 : 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#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum#25] - -(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#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 : 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#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 : 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#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 : 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)] +Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16, rn#17] +Arguments: [avg(_w0#16) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#18], [i_category#3, i_brand#2, cc_name#13, d_year#10] + +(27) Filter [codegen id : 7] +Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16, rn#17, avg_monthly_sales#18] +Condition : ((isnotnull(avg_monthly_sales#18) AND (avg_monthly_sales#18 > 0.000000)) AND CASE WHEN (avg_monthly_sales#18 > 0.000000) THEN ((abs((sum_sales#15 - avg_monthly_sales#18)) / avg_monthly_sales#18) > 0.1000000000000000) END) + +(28) Project [codegen id : 7] +Output [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, avg_monthly_sales#18, rn#17] +Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16, rn#17, avg_monthly_sales#18] + +(29) ReusedExchange [Reuses operator id: 21] +Output [6]: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15] + +(30) CometSort +Input [6]: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15] +Arguments: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15], [i_category#19 ASC NULLS FIRST, i_brand#20 ASC NULLS FIRST, cc_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] + +(31) ColumnarToRow [codegen id : 3] +Input [6]: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15] + +(32) Window +Input [6]: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15] +Arguments: [rank(d_year#22, d_moy#23) windowspecdefinition(i_category#19, i_brand#20, cc_name#21, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#24], [i_category#19, i_brand#20, cc_name#21], [d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] + +(33) Project [codegen id : 4] +Output [5]: [i_category#19, i_brand#20, cc_name#21, sum_sales#15 AS sum_sales#25, rn#24] +Input [7]: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15, rn#24] + +(34) BroadcastExchange +Input [5]: [i_category#19, i_brand#20, cc_name#21, sum_sales#25, rn#24] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=3] + +(35) BroadcastHashJoin [codegen id : 7] +Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#17] +Right keys [4]: [i_category#19, i_brand#20, cc_name#21, (rn#24 + 1)] Join type: Inner Join condition: None -(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] +(36) Project [codegen id : 7] +Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, avg_monthly_sales#18, rn#17, sum_sales#25] +Input [13]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, avg_monthly_sales#18, rn#17, i_category#19, i_brand#20, cc_name#21, sum_sales#25, rn#24] + +(37) ReusedExchange [Reuses operator id: 21] +Output [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#15] -(38) ReusedExchange [Reuses operator id: 31] -Output [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#16] +(38) CometSort +Input [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#15] +Arguments: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#15], [i_category#26 ASC NULLS FIRST, i_brand#27 ASC NULLS FIRST, cc_name#28 ASC NULLS FIRST, d_year#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST] -(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 +(39) ColumnarToRow [codegen id : 5] +Input [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#15] (40) Window -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] +Input [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#15] +Arguments: [rank(d_year#29, d_moy#30) windowspecdefinition(i_category#26, i_brand#27, cc_name#28, d_year#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#31], [i_category#26, i_brand#27, cc_name#28], [d_year#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST] -(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] +(41) Project [codegen id : 6] +Output [5]: [i_category#26, i_brand#27, cc_name#28, sum_sales#15 AS sum_sales#32, rn#31] +Input [7]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#15, rn#31] (42) BroadcastExchange -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] +Input [5]: [i_category#26, i_brand#27, cc_name#28, sum_sales#32, rn#31] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=4] -(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)] +(43) BroadcastHashJoin [codegen id : 7] +Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#17] +Right keys [4]: [i_category#26, i_brand#27, cc_name#28, (rn#31 - 1)] Join type: Inner Join condition: None -(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] +(44) Project [codegen id : 7] +Output [8]: [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#18, sum_sales#15, sum_sales#25 AS psum#33, sum_sales#32 AS nsum#34] +Input [14]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, avg_monthly_sales#18, rn#17, sum_sales#25, i_category#26, i_brand#27, cc_name#28, sum_sales#32, rn#31] (45) TakeOrderedAndProject -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] +Input [8]: [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#18, sum_sales#15, psum#33, nsum#34] +Arguments: 100, [(sum_sales#15 - avg_monthly_sales#18) ASC NULLS FIRST, d_year#10 ASC NULLS FIRST], [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#18, sum_sales#15, psum#33, nsum#34] ===== Subqueries ===== @@ -275,6 +268,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=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/simplified.txt index 968f7e1af..ed9e697f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] - WholeStageCodegen (13) + WholeStageCodegen (7) 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,67 +8,61 @@ 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 (4) + WholeStageCodegen (2) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (3) - Sort [i_category,i_brand,cc_name,d_year,d_moy] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [i_category,i_brand,cc_name] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum,cs_sales_price] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometFilter [i_item_sk,i_brand,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 - CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,cc_name] #1 + CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum,sum(UnscaledValue(cs_sales_price))] + CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum,cs_sales_price] + CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_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_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 + CometFilter [cc_call_center_sk,cc_name] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter BroadcastExchange #7 - WholeStageCodegen (8) + WholeStageCodegen (4) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (7) - Sort [i_category,i_brand,cc_name,d_year,d_moy] + WholeStageCodegen (3) + ColumnarToRow InputAdapter - 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 + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #1 InputAdapter - BroadcastExchange #9 - WholeStageCodegen (12) + BroadcastExchange #8 + WholeStageCodegen (6) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (11) - Sort [i_category,i_brand,cc_name,d_year,d_moy] + WholeStageCodegen (5) + ColumnarToRow InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/explain.txt index fef6345ff..3f4892277 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/explain.txt @@ -1,87 +1,97 @@ == Physical Plan == -TakeOrderedAndProject (83) -+- * HashAggregate (82) - +- Exchange (81) - +- * HashAggregate (80) - +- Union (79) - :- * HashAggregate (68) - : +- Exchange (67) - : +- * HashAggregate (66) - : +- Union (65) - : :- * 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) - : +- * 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) - : +- Exchange (72) - : +- * HashAggregate (71) - : +- * HashAggregate (70) - : +- ReusedExchange (69) - +- * HashAggregate (78) - +- Exchange (77) - +- * HashAggregate (76) - +- * HashAggregate (75) - +- ReusedExchange (74) +* ColumnarToRow (93) ++- CometTakeOrderedAndProject (92) + +- CometHashAggregate (91) + +- CometColumnarExchange (90) + +- RowToColumnar (89) + +- * HashAggregate (88) + +- Union (87) + :- * HashAggregate (70) + : +- * ColumnarToRow (69) + : +- CometColumnarExchange (68) + : +- RowToColumnar (67) + : +- * HashAggregate (66) + : +- Union (65) + : :- * HashAggregate (22) + : : +- * ColumnarToRow (21) + : : +- CometColumnarExchange (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) + : : +- * ColumnarToRow (40) + : : +- CometColumnarExchange (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) + : +- * ColumnarToRow (63) + : +- CometColumnarExchange (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 (78) + : +- * ColumnarToRow (77) + : +- CometColumnarExchange (76) + : +- RowToColumnar (75) + : +- * HashAggregate (74) + : +- * HashAggregate (73) + : +- * ColumnarToRow (72) + : +- ReusedExchange (71) + +- * HashAggregate (86) + +- * ColumnarToRow (85) + +- CometColumnarExchange (84) + +- RowToColumnar (83) + +- * HashAggregate (82) + +- * HashAggregate (81) + +- * ColumnarToRow (80) + +- ReusedExchange (79) (1) Scan parquet spark_catalog.default.store_sales @@ -177,14 +187,14 @@ 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))] -(20) ColumnarToRow [codegen id : 1] +(20) CometColumnarExchange Input [5]: [s_store_id#25, sum#26, sum#27, sum#28, sum#29] +Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(21) Exchange +(21) ColumnarToRow [codegen id : 1] 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] -(22) HashAggregate [codegen id : 2] +(22) HashAggregate [codegen id : 1] 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))] @@ -268,14 +278,14 @@ Input [5]: [sales_price#46, profit#47, return_amt#48, net_loss#49, cp_catalog_pa 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] +(39) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=2] -(40) Exchange +(40) ColumnarToRow [codegen id : 2] 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] +(41) HashAggregate [codegen id : 2] 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))] @@ -378,14 +388,14 @@ Input [5]: [sales_price#83, profit#84, return_amt#85, net_loss#86, web_site_id#1 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) ColumnarToRow [codegen id : 5] +(62) CometColumnarExchange Input [5]: [web_site_id#104, sum#105, sum#106, sum#107, sum#108] +Arguments: hashpartitioning(web_site_id#104, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(63) Exchange +(63) ColumnarToRow [codegen id : 3] 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 : 6] +(64) HashAggregate [codegen id : 3] 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))] @@ -394,133 +404,161 @@ Results [5]: [web channel AS channel#113, concat(web_site, web_site_id#104) AS i (65) Union -(66) HashAggregate [codegen id : 7] +(66) HashAggregate [codegen id : 4] 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 +(67) RowToColumnar 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 : 8] +(68) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=4] + +(69) ColumnarToRow [codegen id : 5] +Input [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] + +(70) HashAggregate [codegen id : 5] 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] +(71) ReusedExchange [Reuses operator id: 68] Output [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] -(70) HashAggregate [codegen id : 16] +(72) ColumnarToRow [codegen id : 10] +Input [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] + +(73) HashAggregate [codegen id : 10] 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] +(74) HashAggregate [codegen id : 10] 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 +(75) RowToColumnar 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 : 17] +(76) CometColumnarExchange +Input [7]: [channel#34, sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150] +Arguments: hashpartitioning(channel#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(77) ColumnarToRow [codegen id : 11] +Input [7]: [channel#34, sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150] + +(78) HashAggregate [codegen id : 11] 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] +(79) ReusedExchange [Reuses operator id: 68] Output [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] -(75) HashAggregate [codegen id : 25] +(80) ColumnarToRow [codegen id : 16] +Input [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] + +(81) 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 [3]: [sum(sales#36)#130 AS sales#158, sum(returns#37)#131 AS returns#159, sum(profit#38)#132 AS profit#160] -(76) HashAggregate [codegen id : 25] +(82) HashAggregate [codegen id : 16] Input [3]: [sales#158, returns#159, profit#160] Keys: [] 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 [6]: [sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172] -(77) Exchange +(83) RowToColumnar +Input [6]: [sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172] + +(84) CometColumnarExchange Input [6]: [sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(78) HashAggregate [codegen id : 26] +(85) ColumnarToRow [codegen id : 17] +Input [6]: [sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172] + +(86) HashAggregate [codegen id : 17] Input [6]: [sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172] Keys: [] 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]: [null AS channel#176, null AS id#177, sum(sales#158)#173 AS sum(sales)#178, sum(returns#159)#174 AS sum(returns)#179, sum(profit#160)#175 AS sum(profit)#180] -(79) Union +(87) Union -(80) HashAggregate [codegen id : 27] +(88) HashAggregate [codegen id : 18] 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#34, id#35, sales#133, returns#134, profit#135] -(81) Exchange +(89) RowToColumnar 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 : 28] +(90) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=7] + +(91) CometHashAggregate 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#34, id#35, sales#133, returns#134, profit#135] -(83) TakeOrderedAndProject +(92) CometTakeOrderedAndProject +Input [5]: [channel#34, id#35, sales#133, returns#134, profit#135] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#34 ASC NULLS FIRST,id#35 ASC NULLS FIRST], output=[channel#34,id#35,sales#133,returns#134,profit#135]), [channel#34, id#35, sales#133, returns#134, profit#135], 100, [channel#34 ASC NULLS FIRST, id#35 ASC NULLS FIRST], [channel#34, id#35, sales#133, returns#134, profit#135] + +(93) ColumnarToRow [codegen id : 19] 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 ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (88) -+- * ColumnarToRow (87) - +- CometProject (86) - +- CometFilter (85) - +- CometScan parquet spark_catalog.default.date_dim (84) +BroadcastExchange (98) ++- * ColumnarToRow (97) + +- CometProject (96) + +- CometFilter (95) + +- CometScan parquet spark_catalog.default.date_dim (94) -(84) Scan parquet spark_catalog.default.date_dim +(94) 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 -(85) CometFilter +(95) 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)) -(86) CometProject +(96) CometProject Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(87) ColumnarToRow [codegen id : 1] +(97) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(88) BroadcastExchange +(98) BroadcastExchange Input [1]: [d_date_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt index 1428c7ba5..00dd3c480 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt @@ -1,124 +1,128 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (28) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Exchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (27) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - 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 (7) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] - CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] - CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - 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,ss_ext_sales_price,ss_net_profit,ss_sold_date_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 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_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 - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #6 - CometFilter [s_store_sk,s_store_id] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] +WholeStageCodegen (19) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [channel,id,sales,returns,profit] + CometColumnarExchange [channel,id,sales,returns,profit] #1 + RowToColumnar + WholeStageCodegen (18) + HashAggregate [channel,id,sales,returns,profit] + InputAdapter + Union + WholeStageCodegen (5) + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel,id] #2 + RowToColumnar 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] + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange [cp_catalog_page_id] #7 + Union + WholeStageCodegen (1) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [s_store_id] #3 + CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] + CometProject [sales_price,profit,return_amt,net_loss,s_store_id] + CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] + CometProject [store_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + 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,ss_ext_sales_price,ss_net_profit,ss_sold_date_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 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_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 + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [s_store_sk,s_store_id] #6 + CometFilter [s_store_sk,s_store_id] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + WholeStageCodegen (2) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [cp_catalog_page_id] #7 + CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] + CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [page_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + 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,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_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 + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #8 + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] - CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - 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,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_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 - ReusedExchange [d_date_sk] #5 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #8 - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - 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] #9 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] - CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] - CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - 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,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - 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,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] - CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #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_web_site_sk,ws_order_number,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - ReusedExchange [d_date_sk] #5 - CometBroadcastExchange [web_site_sk,web_site_id] #11 - CometFilter [web_site_sk,web_site_id] - 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] #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 (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 #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 - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [web_site_id] #9 + CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] + CometProject [sales_price,profit,return_amt,net_loss,web_site_id] + CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] + CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + 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,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + 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,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] + CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #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_web_site_sk,ws_order_number,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange [web_site_sk,web_site_id] #11 + CometFilter [web_site_sk,web_site_id] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + WholeStageCodegen (11) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel] #12 + RowToColumnar + WholeStageCodegen (10) + 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] + ColumnarToRow + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + WholeStageCodegen (17) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange #13 + RowToColumnar + WholeStageCodegen (16) + 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] + ColumnarToRow + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/explain.txt index fc156f98e..b662f22fb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/explain.txt @@ -1,44 +1,42 @@ == Physical Plan == -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) +* ColumnarToRow (38) ++- CometTakeOrderedAndProject (37) + +- CometFilter (36) + +- CometHashAggregate (35) + +- CometColumnarExchange (34) + +- CometHashAggregate (33) + +- CometProject (32) + +- CometBroadcastHashJoin (31) + :- 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) + +- CometBroadcastExchange (30) + +- CometProject (29) + +- CometBroadcastHashJoin (28) + :- CometFilter (20) + : +- CometScan parquet spark_catalog.default.item (19) + +- CometBroadcastExchange (27) + +- CometFilter (26) + +- CometHashAggregate (25) + +- CometColumnarExchange (24) + +- CometHashAggregate (23) + +- CometFilter (22) + +- CometScan parquet spark_catalog.default.item (21) (1) Scan parquet spark_catalog.default.customer_address @@ -129,186 +127,170 @@ Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight 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] - -(20) Scan parquet spark_catalog.default.item +(19) 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 -(21) CometFilter +(20) 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)) -(22) ColumnarToRow [codegen id : 3] -Input [3]: [i_item_sk#13, i_current_price#14, i_category#15] - -(23) Scan parquet spark_catalog.default.item +(21) 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 -(24) CometFilter +(22) CometFilter Input [2]: [i_current_price#16, i_category#17] Condition : isnotnull(i_category#17) -(25) CometHashAggregate +(23) CometHashAggregate Input [2]: [i_current_price#16, i_category#17] Keys [1]: [i_category#17] Functions [1]: [partial_avg(UnscaledValue(i_current_price#16))] -(26) ColumnarToRow [codegen id : 1] +(24) CometColumnarExchange Input [3]: [i_category#17, sum#18, count#19] +Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(27) Exchange -Input [3]: [i_category#17, sum#18, count#19] -Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, [plan_id=1] - -(28) HashAggregate [codegen id : 2] +(25) CometHashAggregate 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] -(29) Filter [codegen id : 2] -Input [2]: [avg(i_current_price)#21, i_category#17] -Condition : isnotnull(avg(i_current_price)#21) +(26) CometFilter +Input [2]: [avg(i_current_price)#20, i_category#17] +Condition : isnotnull(avg(i_current_price)#20) -(30) BroadcastExchange -Input [2]: [avg(i_current_price)#21, i_category#17] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=2] +(27) CometBroadcastExchange +Input [2]: [avg(i_current_price)#20, i_category#17] +Arguments: [avg(i_current_price)#20, i_category#17] -(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#14 as decimal(14,7)) > (1.2 * avg(i_current_price)#21)) +(28) CometBroadcastHashJoin +Left output [3]: [i_item_sk#13, i_current_price#14, i_category#15] +Right output [2]: [avg(i_current_price)#20, i_category#17] +Arguments: [i_category#15], [i_category#17], Inner, (cast(i_current_price#14 as decimal(14,7)) > (1.2 * avg(i_current_price)#20)), BuildRight -(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] +(29) CometProject +Input [5]: [i_item_sk#13, i_current_price#14, i_category#15, avg(i_current_price)#20, i_category#17] +Arguments: [i_item_sk#13], [i_item_sk#13] -(33) BroadcastExchange +(30) CometBroadcastExchange Input [1]: [i_item_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: [i_item_sk#13] -(34) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#5] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None +(31) CometBroadcastHashJoin +Left output [2]: [ca_state#2, ss_item_sk#5] +Right output [1]: [i_item_sk#13] +Arguments: [ss_item_sk#5], [i_item_sk#13], Inner, BuildRight -(35) Project [codegen id : 4] -Output [1]: [ca_state#2] +(32) CometProject Input [3]: [ca_state#2, ss_item_sk#5, i_item_sk#13] +Arguments: [ca_state#2], [ca_state#2] -(36) HashAggregate [codegen id : 4] +(33) CometHashAggregate Input [1]: [ca_state#2] Keys [1]: [ca_state#2] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#22] -Results [2]: [ca_state#2, count#23] -(37) Exchange -Input [2]: [ca_state#2, count#23] -Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(34) CometColumnarExchange +Input [2]: [ca_state#2, count#21] +Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(38) HashAggregate [codegen id : 5] -Input [2]: [ca_state#2, count#23] +(35) CometHashAggregate +Input [2]: [ca_state#2, count#21] Keys [1]: [ca_state#2] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#24] -Results [3]: [ca_state#2 AS state#25, count(1)#24 AS cnt#26, ca_state#2] -(39) Filter [codegen id : 5] -Input [3]: [state#25, cnt#26, ca_state#2] -Condition : (cnt#26 >= 10) +(36) CometFilter +Input [3]: [state#22, cnt#23, ca_state#2] +Condition : (cnt#23 >= 10) -(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] +(37) CometTakeOrderedAndProject +Input [3]: [state#22, cnt#23, ca_state#2] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[cnt#23 ASC NULLS FIRST,ca_state#2 ASC NULLS FIRST], output=[state#22,cnt#23]), [state#22, cnt#23], 100, [cnt#23 ASC NULLS FIRST, ca_state#2 ASC NULLS FIRST], [state#22, cnt#23] + +(38) ColumnarToRow [codegen id : 1] +Input [2]: [state#22, cnt#23] ===== Subqueries ===== 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) +BroadcastExchange (43) ++- * ColumnarToRow (42) + +- CometProject (41) + +- CometFilter (40) + +- CometScan parquet spark_catalog.default.date_dim (39) -(41) Scan parquet spark_catalog.default.date_dim +(39) 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 -(42) CometFilter +(40) 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)) -(43) CometProject +(41) CometProject Input [2]: [d_date_sk#9, d_month_seq#10] Arguments: [d_date_sk#9], [d_date_sk#9] -(44) ColumnarToRow [codegen id : 1] +(42) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(45) BroadcastExchange +(43) BroadcastExchange Input [1]: [d_date_sk#9] -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 = 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) +Subquery:2 Hosting operator id = 40 Hosting Expression = Subquery scalar-subquery#11, [id=#12] +* ColumnarToRow (50) ++- CometHashAggregate (49) + +- CometColumnarExchange (48) + +- CometHashAggregate (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan parquet spark_catalog.default.date_dim (44) -(46) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#27, d_year#28, d_moy#29] +(44) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_month_seq#24, d_year#25, d_moy#26] 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 -(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)) +(45) CometFilter +Input [3]: [d_month_seq#24, d_year#25, d_moy#26] +Condition : (((isnotnull(d_year#25) AND isnotnull(d_moy#26)) AND (d_year#25 = 2000)) AND (d_moy#26 = 1)) -(48) CometProject -Input [3]: [d_month_seq#27, d_year#28, d_moy#29] -Arguments: [d_month_seq#27], [d_month_seq#27] +(46) CometProject +Input [3]: [d_month_seq#24, d_year#25, d_moy#26] +Arguments: [d_month_seq#24], [d_month_seq#24] -(49) CometHashAggregate -Input [1]: [d_month_seq#27] -Keys [1]: [d_month_seq#27] +(47) CometHashAggregate +Input [1]: [d_month_seq#24] +Keys [1]: [d_month_seq#24] Functions: [] -(50) ColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#27] - -(51) Exchange -Input [1]: [d_month_seq#27] -Arguments: hashpartitioning(d_month_seq#27, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(48) CometColumnarExchange +Input [1]: [d_month_seq#24] +Arguments: hashpartitioning(d_month_seq#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(52) HashAggregate [codegen id : 2] -Input [1]: [d_month_seq#27] -Keys [1]: [d_month_seq#27] +(49) CometHashAggregate +Input [1]: [d_month_seq#24] +Keys [1]: [d_month_seq#24] Functions: [] -Aggregate Attributes: [] -Results [1]: [d_month_seq#27] + +(50) ColumnarToRow [codegen id : 1] +Input [1]: [d_month_seq#24] 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-spark3_5/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/simplified.txt index cfb74b3b2..b95744087 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/simplified.txt @@ -1,73 +1,59 @@ -TakeOrderedAndProject [cnt,ca_state,state] - WholeStageCodegen (5) - Filter [cnt] - HashAggregate [ca_state,count] [count(1),state,cnt,count] - InputAdapter - Exchange [ca_state] #1 - WholeStageCodegen (4) - HashAggregate [ca_state] [count,count] - Project [ca_state] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometProject [ca_state,ss_item_sk] - CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] - CometProject [ca_state,ss_item_sk,ss_sold_date_sk] - CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] - CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_customer_sk,ss_sold_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 #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [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_month_seq,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 [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [d_month_seq] #2 - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - Project [i_item_sk] - BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_current_price,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - InputAdapter - 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] #9 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_category,sum,count,i_current_price] - CometFilter [i_current_price,i_category] - CometScan parquet spark_catalog.default.item [i_current_price,i_category] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [state,cnt,ca_state] + CometFilter [state,cnt,ca_state] + CometHashAggregate [state,cnt,ca_state,count,count(1)] + CometColumnarExchange [ca_state] #1 + CometHashAggregate [ca_state,count] + CometProject [ca_state] + CometBroadcastHashJoin [ca_state,ss_item_sk,i_item_sk] + CometProject [ca_state,ss_item_sk] + CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] + CometProject [ca_state,ss_item_sk,ss_sold_date_sk] + CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] + CometProject [ca_state,c_customer_sk] + CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_customer_sk,ss_sold_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 #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + Subquery #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [d_month_seq] + CometColumnarExchange [d_month_seq] #5 + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_month_seq,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 [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + ReusedSubquery [d_month_seq] #2 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk] #7 + CometProject [i_item_sk] + CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] + CometFilter [i_item_sk,i_current_price,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] + CometBroadcastExchange [avg(i_current_price),i_category] #8 + CometFilter [avg(i_current_price),i_category] + CometHashAggregate [avg(i_current_price),i_category,sum,count,avg(UnscaledValue(i_current_price))] + CometColumnarExchange [i_category] #9 + CometHashAggregate [i_category,sum,count,i_current_price] + CometFilter [i_current_price,i_category] + CometScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/explain.txt index 4088fb409..174590641 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/explain.txt @@ -1,185 +1,180 @@ == Physical Plan == -* Sort (181) -+- Exchange (180) - +- * Project (179) - +- * SortMergeJoin Inner (178) - :- * Sort (110) - : +- Exchange (109) - : +- * HashAggregate (108) - : +- * HashAggregate (107) - : +- * Project (106) - : +- * BroadcastHashJoin Inner BuildRight (105) - : :- * Project (99) - : : +- * BroadcastHashJoin Inner BuildRight (98) - : : :- * Project (96) - : : : +- * BroadcastHashJoin Inner BuildRight (95) - : : : :- * Project (90) - : : : : +- * BroadcastHashJoin Inner BuildRight (89) - : : : : :- * Project (87) - : : : : : +- * BroadcastHashJoin Inner BuildRight (86) - : : : : : :- * Project (81) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (80) - : : : : : : :- * Project (78) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (77) - : : : : : : : :- * Project (72) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (71) - : : : : : : : : :- * Project (66) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (65) - : : : : : : : : : :- * Project (63) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (62) - : : : : : : : : : : :- * Project (57) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (56) - : : : : : : : : : : : :- * Project (54) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (53) - : : : : : : : : : : : : :- * Project (48) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (47) - : : : : : : : : : : : : : :- * Project (42) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : : : : : : : : : : : :- * Project (36) - : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (35) - : : : : : : : : : : : : : : : :- * Project (33) - : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (32) - : : : : : : : : : : : : : : : : :- * Sort (11) - : : : : : : : : : : : : : : : : : +- Exchange (10) - : : : : : : : : : : : : : : : : : +- * ColumnarToRow (9) - : : : : : : : : : : : : : : : : : +- CometProject (8) - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) - : : : : : : : : : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- CometProject (6) - : : : : : : : : : : : : : : : : : +- CometFilter (5) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : : : : : : : : : : : : : : : +- * Sort (31) - : : : : : : : : : : : : : : : : +- * Project (30) - : : : : : : : : : : : : : : : : +- * Filter (29) - : : : : : : : : : : : : : : : : +- * HashAggregate (28) - : : : : : : : : : : : : : : : : +- Exchange (27) - : : : : : : : : : : : : : : : : +- * HashAggregate (26) - : : : : : : : : : : : : : : : : +- * Project (25) - : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (24) - : : : : : : : : : : : : : : : : :- * Sort (17) - : : : : : : : : : : : : : : : : : +- Exchange (16) - : : : : : : : : : : : : : : : : : +- * ColumnarToRow (15) - : : : : : : : : : : : : : : : : : +- CometProject (14) - : : : : : : : : : : : : : : : : : +- CometFilter (13) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (12) - : : : : : : : : : : : : : : : : +- * Sort (23) - : : : : : : : : : : : : : : : : +- Exchange (22) - : : : : : : : : : : : : : : : : +- * ColumnarToRow (21) - : : : : : : : : : : : : : : : : +- CometProject (20) - : : : : : : : : : : : : : : : : +- CometFilter (19) - : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (18) - : : : : : : : : : : : : : : : +- ReusedExchange (34) - : : : : : : : : : : : : : : +- BroadcastExchange (40) - : : : : : : : : : : : : : : +- * ColumnarToRow (39) - : : : : : : : : : : : : : : +- CometFilter (38) - : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (37) - : : : : : : : : : : : : : +- BroadcastExchange (46) - : : : : : : : : : : : : : +- * ColumnarToRow (45) - : : : : : : : : : : : : : +- CometFilter (44) - : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (43) - : : : : : : : : : : : : +- BroadcastExchange (52) - : : : : : : : : : : : : +- * ColumnarToRow (51) - : : : : : : : : : : : : +- CometFilter (50) - : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (49) - : : : : : : : : : : : +- ReusedExchange (55) - : : : : : : : : : : +- BroadcastExchange (61) - : : : : : : : : : : +- * ColumnarToRow (60) - : : : : : : : : : : +- CometFilter (59) - : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (58) - : : : : : : : : : +- ReusedExchange (64) - : : : : : : : : +- BroadcastExchange (70) - : : : : : : : : +- * ColumnarToRow (69) - : : : : : : : : +- CometFilter (68) - : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (67) - : : : : : : : +- BroadcastExchange (76) - : : : : : : : +- * ColumnarToRow (75) - : : : : : : : +- CometFilter (74) - : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (73) - : : : : : : +- ReusedExchange (79) - : : : : : +- BroadcastExchange (85) - : : : : : +- * ColumnarToRow (84) - : : : : : +- CometFilter (83) - : : : : : +- CometScan parquet spark_catalog.default.customer_address (82) - : : : : +- ReusedExchange (88) - : : : +- BroadcastExchange (94) - : : : +- * ColumnarToRow (93) - : : : +- CometFilter (92) - : : : +- CometScan parquet spark_catalog.default.income_band (91) - : : +- ReusedExchange (97) - : +- BroadcastExchange (104) - : +- * ColumnarToRow (103) - : +- CometProject (102) - : +- CometFilter (101) - : +- CometScan parquet spark_catalog.default.item (100) - +- * Sort (177) - +- Exchange (176) - +- * HashAggregate (175) - +- * HashAggregate (174) - +- * Project (173) - +- * BroadcastHashJoin Inner BuildRight (172) - :- * Project (170) - : +- * BroadcastHashJoin Inner BuildRight (169) - : :- * Project (167) - : : +- * BroadcastHashJoin Inner BuildRight (166) - : : :- * Project (164) - : : : +- * BroadcastHashJoin Inner BuildRight (163) - : : : :- * Project (161) - : : : : +- * BroadcastHashJoin Inner BuildRight (160) - : : : : :- * Project (158) - : : : : : +- * BroadcastHashJoin Inner BuildRight (157) - : : : : : :- * Project (155) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (154) - : : : : : : :- * Project (152) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (151) - : : : : : : : :- * Project (149) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (148) - : : : : : : : : :- * Project (146) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (145) - : : : : : : : : : :- * Project (143) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (142) - : : : : : : : : : : :- * Project (140) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (139) - : : : : : : : : : : : :- * Project (137) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (136) - : : : : : : : : : : : : :- * Project (134) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (133) - : : : : : : : : : : : : : :- * Project (131) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (130) - : : : : : : : : : : : : : : :- * Project (128) - : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (127) - : : : : : : : : : : : : : : : :- * Sort (121) - : : : : : : : : : : : : : : : : +- Exchange (120) - : : : : : : : : : : : : : : : : +- * ColumnarToRow (119) - : : : : : : : : : : : : : : : : +- CometProject (118) - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (117) - : : : : : : : : : : : : : : : : :- CometBroadcastExchange (113) - : : : : : : : : : : : : : : : : : +- CometFilter (112) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (111) - : : : : : : : : : : : : : : : : +- CometProject (116) - : : : : : : : : : : : : : : : : +- CometFilter (115) - : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (114) - : : : : : : : : : : : : : : : +- * Sort (126) - : : : : : : : : : : : : : : : +- * Project (125) - : : : : : : : : : : : : : : : +- * Filter (124) - : : : : : : : : : : : : : : : +- * HashAggregate (123) - : : : : : : : : : : : : : : : +- ReusedExchange (122) - : : : : : : : : : : : : : : +- ReusedExchange (129) - : : : : : : : : : : : : : +- ReusedExchange (132) - : : : : : : : : : : : : +- ReusedExchange (135) - : : : : : : : : : : : +- ReusedExchange (138) - : : : : : : : : : : +- ReusedExchange (141) - : : : : : : : : : +- ReusedExchange (144) - : : : : : : : : +- ReusedExchange (147) - : : : : : : : +- ReusedExchange (150) - : : : : : : +- ReusedExchange (153) - : : : : : +- ReusedExchange (156) - : : : : +- ReusedExchange (159) - : : : +- ReusedExchange (162) - : : +- ReusedExchange (165) - : +- ReusedExchange (168) - +- ReusedExchange (171) +* ColumnarToRow (176) ++- CometSort (175) + +- CometColumnarExchange (174) + +- RowToColumnar (173) + +- * Project (172) + +- * SortMergeJoin Inner (171) + :- * ColumnarToRow (101) + : +- CometSort (100) + : +- CometColumnarExchange (99) + : +- CometHashAggregate (98) + : +- CometHashAggregate (97) + : +- CometProject (96) + : +- CometBroadcastHashJoin (95) + : :- CometProject (90) + : : +- CometBroadcastHashJoin (89) + : : :- CometProject (87) + : : : +- CometBroadcastHashJoin (86) + : : : :- CometProject (82) + : : : : +- CometBroadcastHashJoin (81) + : : : : :- CometProject (79) + : : : : : +- CometBroadcastHashJoin (78) + : : : : : :- CometProject (74) + : : : : : : +- CometBroadcastHashJoin (73) + : : : : : : :- CometProject (71) + : : : : : : : +- CometBroadcastHashJoin (70) + : : : : : : : :- CometProject (66) + : : : : : : : : +- CometBroadcastHashJoin (65) + : : : : : : : : :- CometProject (61) + : : : : : : : : : +- CometBroadcastHashJoin (60) + : : : : : : : : : :- CometProject (58) + : : : : : : : : : : +- CometBroadcastHashJoin (57) + : : : : : : : : : : :- CometProject (53) + : : : : : : : : : : : +- CometBroadcastHashJoin (52) + : : : : : : : : : : : :- CometProject (50) + : : : : : : : : : : : : +- CometBroadcastHashJoin (49) + : : : : : : : : : : : : :- CometProject (45) + : : : : : : : : : : : : : +- CometBroadcastHashJoin (44) + : : : : : : : : : : : : : :- CometProject (40) + : : : : : : : : : : : : : : +- CometBroadcastHashJoin (39) + : : : : : : : : : : : : : : :- CometProject (35) + : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (34) + : : : : : : : : : : : : : : : :- CometProject (30) + : : : : : : : : : : : : : : : : +- CometSortMergeJoin (29) + : : : : : : : : : : : : : : : : :- CometSort (10) + : : : : : : : : : : : : : : : : : +- CometColumnarExchange (9) + : : : : : : : : : : : : : : : : : +- CometProject (8) + : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) + : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) + : : : : : : : : : : : : : : : : : : +- CometFilter (2) + : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : : : : : : : : : : : : +- CometProject (6) + : : : : : : : : : : : : : : : : : +- CometFilter (5) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : : : : : : : : : : : : : : : +- CometSort (28) + : : : : : : : : : : : : : : : : +- CometProject (27) + : : : : : : : : : : : : : : : : +- CometFilter (26) + : : : : : : : : : : : : : : : : +- CometHashAggregate (25) + : : : : : : : : : : : : : : : : +- CometColumnarExchange (24) + : : : : : : : : : : : : : : : : +- CometHashAggregate (23) + : : : : : : : : : : : : : : : : +- CometProject (22) + : : : : : : : : : : : : : : : : +- CometSortMergeJoin (21) + : : : : : : : : : : : : : : : : :- CometSort (15) + : : : : : : : : : : : : : : : : : +- CometColumnarExchange (14) + : : : : : : : : : : : : : : : : : +- CometProject (13) + : : : : : : : : : : : : : : : : : +- CometFilter (12) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (11) + : : : : : : : : : : : : : : : : +- CometSort (20) + : : : : : : : : : : : : : : : : +- CometColumnarExchange (19) + : : : : : : : : : : : : : : : : +- CometProject (18) + : : : : : : : : : : : : : : : : +- CometFilter (17) + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (16) + : : : : : : : : : : : : : : : +- CometBroadcastExchange (33) + : : : : : : : : : : : : : : : +- CometFilter (32) + : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (31) + : : : : : : : : : : : : : : +- CometBroadcastExchange (38) + : : : : : : : : : : : : : : +- CometFilter (37) + : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (36) + : : : : : : : : : : : : : +- CometBroadcastExchange (43) + : : : : : : : : : : : : : +- CometFilter (42) + : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (41) + : : : : : : : : : : : : +- CometBroadcastExchange (48) + : : : : : : : : : : : : +- CometFilter (47) + : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (46) + : : : : : : : : : : : +- ReusedExchange (51) + : : : : : : : : : : +- CometBroadcastExchange (56) + : : : : : : : : : : +- CometFilter (55) + : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (54) + : : : : : : : : : +- ReusedExchange (59) + : : : : : : : : +- CometBroadcastExchange (64) + : : : : : : : : +- CometFilter (63) + : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (62) + : : : : : : : +- CometBroadcastExchange (69) + : : : : : : : +- CometFilter (68) + : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (67) + : : : : : : +- ReusedExchange (72) + : : : : : +- CometBroadcastExchange (77) + : : : : : +- CometFilter (76) + : : : : : +- CometScan parquet spark_catalog.default.customer_address (75) + : : : : +- ReusedExchange (80) + : : : +- CometBroadcastExchange (85) + : : : +- CometFilter (84) + : : : +- CometScan parquet spark_catalog.default.income_band (83) + : : +- ReusedExchange (88) + : +- CometBroadcastExchange (94) + : +- CometProject (93) + : +- CometFilter (92) + : +- CometScan parquet spark_catalog.default.item (91) + +- * ColumnarToRow (170) + +- CometSort (169) + +- CometColumnarExchange (168) + +- CometHashAggregate (167) + +- CometHashAggregate (166) + +- CometProject (165) + +- CometBroadcastHashJoin (164) + :- CometProject (162) + : +- CometBroadcastHashJoin (161) + : :- CometProject (159) + : : +- CometBroadcastHashJoin (158) + : : :- CometProject (156) + : : : +- CometBroadcastHashJoin (155) + : : : :- CometProject (153) + : : : : +- CometBroadcastHashJoin (152) + : : : : :- CometProject (150) + : : : : : +- CometBroadcastHashJoin (149) + : : : : : :- CometProject (147) + : : : : : : +- CometBroadcastHashJoin (146) + : : : : : : :- CometProject (144) + : : : : : : : +- CometBroadcastHashJoin (143) + : : : : : : : :- CometProject (141) + : : : : : : : : +- CometBroadcastHashJoin (140) + : : : : : : : : :- CometProject (138) + : : : : : : : : : +- CometBroadcastHashJoin (137) + : : : : : : : : : :- CometProject (135) + : : : : : : : : : : +- CometBroadcastHashJoin (134) + : : : : : : : : : : :- CometProject (132) + : : : : : : : : : : : +- CometBroadcastHashJoin (131) + : : : : : : : : : : : :- CometProject (129) + : : : : : : : : : : : : +- CometBroadcastHashJoin (128) + : : : : : : : : : : : : :- CometProject (126) + : : : : : : : : : : : : : +- CometBroadcastHashJoin (125) + : : : : : : : : : : : : : :- CometProject (123) + : : : : : : : : : : : : : : +- CometBroadcastHashJoin (122) + : : : : : : : : : : : : : : :- CometProject (118) + : : : : : : : : : : : : : : : +- CometSortMergeJoin (117) + : : : : : : : : : : : : : : : :- CometSort (111) + : : : : : : : : : : : : : : : : +- CometColumnarExchange (110) + : : : : : : : : : : : : : : : : +- CometProject (109) + : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (108) + : : : : : : : : : : : : : : : : :- CometBroadcastExchange (104) + : : : : : : : : : : : : : : : : : +- CometFilter (103) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (102) + : : : : : : : : : : : : : : : : +- CometProject (107) + : : : : : : : : : : : : : : : : +- CometFilter (106) + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (105) + : : : : : : : : : : : : : : : +- CometSort (116) + : : : : : : : : : : : : : : : +- CometProject (115) + : : : : : : : : : : : : : : : +- CometFilter (114) + : : : : : : : : : : : : : : : +- CometHashAggregate (113) + : : : : : : : : : : : : : : : +- ReusedExchange (112) + : : : : : : : : : : : : : : +- CometBroadcastExchange (121) + : : : : : : : : : : : : : : +- CometFilter (120) + : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (119) + : : : : : : : : : : : : : +- ReusedExchange (124) + : : : : : : : : : : : : +- ReusedExchange (127) + : : : : : : : : : : : +- ReusedExchange (130) + : : : : : : : : : : +- ReusedExchange (133) + : : : : : : : : : +- ReusedExchange (136) + : : : : : : : : +- ReusedExchange (139) + : : : : : : : +- ReusedExchange (142) + : : : : : : +- ReusedExchange (145) + : : : : : +- ReusedExchange (148) + : : : : +- ReusedExchange (151) + : : : +- ReusedExchange (154) + : : +- ReusedExchange (157) + : +- ReusedExchange (160) + +- ReusedExchange (163) (1) Scan parquet spark_catalog.default.store_sales @@ -222,843 +217,793 @@ Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number# 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] Arguments: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -(9) ColumnarToRow [codegen id : 1] +(9) CometColumnarExchange Input [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(10) Exchange +(10) CometSort Input [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1 ASC NULLS FIRST] -(11) Sort [codegen id : 2] -Input [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 - -(12) Scan parquet spark_catalog.default.catalog_sales +(11) Scan parquet spark_catalog.default.catalog_sales Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_order_number)] ReadSchema: struct -(13) CometFilter +(12) CometFilter Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) -(14) CometProject +(13) CometProject Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -(15) ColumnarToRow [codegen id : 3] -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] - -(16) Exchange +(14) CometColumnarExchange Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(17) Sort [codegen id : 4] +(15) CometSort Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST], false, 0 +Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST] -(18) Scan parquet spark_catalog.default.catalog_returns +(16) Scan parquet spark_catalog.default.catalog_returns Output [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(19) CometFilter +(17) CometFilter Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) -(20) CometProject +(18) CometProject Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -(21) ColumnarToRow [codegen id : 5] +(19) CometColumnarExchange Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) Exchange +(20) CometSort Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST] -(23) Sort [codegen id : 6] -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST], false, 0 +(21) CometSortMergeJoin +Left output [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] +Right output [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number#22], Inner -(24) SortMergeJoin [codegen id : 7] -Left keys [2]: [cs_item_sk#17, cs_order_number#18] -Right keys [2]: [cr_item_sk#21, cr_order_number#22] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 7] -Output [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +(22) CometProject Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +Arguments: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -(26) HashAggregate [codegen id : 7] +(23) CometHashAggregate Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Keys [1]: [cs_item_sk#17] Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] -Aggregate Attributes [3]: [sum#27, sum#28, isEmpty#29] -Results [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] -(27) Exchange -Input [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(24) CometColumnarExchange +Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] +Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(28) HashAggregate [codegen id : 8] -Input [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] +(25) CometHashAggregate +Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] Keys [1]: [cs_item_sk#17] Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#19))#33, sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))#34] -Results [3]: [cs_item_sk#17, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#19))#33,17,2) AS sale#35, sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))#34 AS refund#36] -(29) Filter [codegen id : 8] -Input [3]: [cs_item_sk#17, sale#35, refund#36] -Condition : ((isnotnull(sale#35) AND isnotnull(refund#36)) AND (cast(sale#35 as decimal(21,2)) > (2 * refund#36))) +(26) CometFilter +Input [3]: [cs_item_sk#17, sale#30, refund#31] +Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) -(30) Project [codegen id : 8] -Output [1]: [cs_item_sk#17] -Input [3]: [cs_item_sk#17, sale#35, refund#36] +(27) CometProject +Input [3]: [cs_item_sk#17, sale#30, refund#31] +Arguments: [cs_item_sk#17], [cs_item_sk#17] -(31) Sort [codegen id : 8] +(28) CometSort Input [1]: [cs_item_sk#17] -Arguments: [cs_item_sk#17 ASC NULLS FIRST], false, 0 +Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] -(32) SortMergeJoin [codegen id : 24] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [cs_item_sk#17] -Join type: Inner -Join condition: None +(29) CometSortMergeJoin +Left output [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Right output [1]: [cs_item_sk#17] +Arguments: [ss_item_sk#1], [cs_item_sk#17], Inner -(33) Project [codegen id : 24] -Output [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +(30) CometProject Input [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#17] +Arguments: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] + +(31) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#32, d_year#33] +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 -(34) ReusedExchange [Reuses operator id: 185] -Output [2]: [d_date_sk#37, d_year#38] +(32) CometFilter +Input [2]: [d_date_sk#32, d_year#33] +Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(35) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#37] -Join type: Inner -Join condition: None +(33) CometBroadcastExchange +Input [2]: [d_date_sk#32, d_year#33] +Arguments: [d_date_sk#32, d_year#33] + +(34) CometBroadcastHashJoin +Left output [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Right output [2]: [d_date_sk#32, d_year#33] +Arguments: [ss_sold_date_sk#12], [d_date_sk#32], Inner, BuildRight -(36) Project [codegen id : 24] -Output [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38] -Input [13]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#37, d_year#38] +(35) CometProject +Input [13]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#32, d_year#33] +Arguments: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33], [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] -(37) Scan parquet spark_catalog.default.store -Output [3]: [s_store_sk#39, s_store_name#40, s_zip#41] +(36) Scan parquet spark_catalog.default.store +Output [3]: [s_store_sk#34, s_store_name#35, s_zip#36] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_zip)] ReadSchema: struct -(38) CometFilter -Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] -Condition : ((isnotnull(s_store_sk#39) AND isnotnull(s_store_name#40)) AND isnotnull(s_zip#41)) +(37) CometFilter +Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] +Condition : ((isnotnull(s_store_sk#34) AND isnotnull(s_store_name#35)) AND isnotnull(s_zip#36)) -(39) ColumnarToRow [codegen id : 10] -Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] +(38) CometBroadcastExchange +Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] +Arguments: [s_store_sk#34, s_store_name#35, s_zip#36] -(40) BroadcastExchange -Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +(39) CometBroadcastHashJoin +Left output [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] +Right output [3]: [s_store_sk#34, s_store_name#35, s_zip#36] +Arguments: [ss_store_sk#6], [s_store_sk#34], Inner, BuildRight -(41) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_store_sk#6] -Right keys [1]: [s_store_sk#39] -Join type: Inner -Join condition: None +(40) 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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_sk#34, s_store_name#35, s_zip#36] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36] -(42) Project [codegen id : 24] -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41] -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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_sk#39, s_store_name#40, s_zip#41] - -(43) Scan parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] +(41) Scan parquet spark_catalog.default.customer +Output [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(44) CometFilter -Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] -Condition : (((((isnotnull(c_customer_sk#42) AND isnotnull(c_first_sales_date_sk#47)) AND isnotnull(c_first_shipto_date_sk#46)) AND isnotnull(c_current_cdemo_sk#43)) AND isnotnull(c_current_hdemo_sk#44)) AND isnotnull(c_current_addr_sk#45)) - -(45) ColumnarToRow [codegen id : 11] -Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] +(42) CometFilter +Input [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +Condition : (((((isnotnull(c_customer_sk#37) AND isnotnull(c_first_sales_date_sk#42)) AND isnotnull(c_first_shipto_date_sk#41)) AND isnotnull(c_current_cdemo_sk#38)) AND isnotnull(c_current_hdemo_sk#39)) AND isnotnull(c_current_addr_sk#40)) -(46) BroadcastExchange -Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +(43) CometBroadcastExchange +Input [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +Arguments: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] -(47) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#42] -Join type: Inner -Join condition: None +(44) 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_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36] +Right output [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +Arguments: [ss_customer_sk#2], [c_customer_sk#37], Inner, BuildRight -(48) Project [codegen id : 24] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] +(45) CometProject +Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] -(49) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#48, d_year#49] +(46) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#43, d_year#44] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct -(50) CometFilter -Input [2]: [d_date_sk#48, d_year#49] -Condition : isnotnull(d_date_sk#48) +(47) CometFilter +Input [2]: [d_date_sk#43, d_year#44] +Condition : isnotnull(d_date_sk#43) -(51) ColumnarToRow [codegen id : 12] -Input [2]: [d_date_sk#48, d_year#49] +(48) CometBroadcastExchange +Input [2]: [d_date_sk#43, d_year#44] +Arguments: [d_date_sk#43, d_year#44] -(52) BroadcastExchange -Input [2]: [d_date_sk#48, d_year#49] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] - -(53) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [c_first_sales_date_sk#47] -Right keys [1]: [d_date_sk#48] -Join type: Inner -Join condition: None +(49) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +Right output [2]: [d_date_sk#43, d_year#44] +Arguments: [c_first_sales_date_sk#42], [d_date_sk#43], Inner, BuildRight -(54) Project [codegen id : 24] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, d_year#49] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47, d_date_sk#48, d_year#49] +(50) CometProject +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42, d_date_sk#43, d_year#44] +Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, d_year#44], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, d_year#44] -(55) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#50, d_year#51] +(51) ReusedExchange [Reuses operator id: 48] +Output [2]: [d_date_sk#45, d_year#46] -(56) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [c_first_shipto_date_sk#46] -Right keys [1]: [d_date_sk#50] -Join type: Inner -Join condition: None +(52) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, d_year#44] +Right output [2]: [d_date_sk#45, d_year#46] +Arguments: [c_first_shipto_date_sk#41], [d_date_sk#45], Inner, BuildRight -(57) Project [codegen id : 24] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, d_year#49, d_date_sk#50, d_year#51] +(53) CometProject +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, d_year#44, d_date_sk#45, d_year#46] +Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] -(58) Scan parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#52, cd_marital_status#53] +(54) Scan parquet spark_catalog.default.customer_demographics +Output [2]: [cd_demo_sk#47, cd_marital_status#48] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status)] ReadSchema: struct -(59) CometFilter -Input [2]: [cd_demo_sk#52, cd_marital_status#53] -Condition : (isnotnull(cd_demo_sk#52) AND isnotnull(cd_marital_status#53)) - -(60) ColumnarToRow [codegen id : 14] -Input [2]: [cd_demo_sk#52, cd_marital_status#53] +(55) CometFilter +Input [2]: [cd_demo_sk#47, cd_marital_status#48] +Condition : (isnotnull(cd_demo_sk#47) AND isnotnull(cd_marital_status#48)) -(61) BroadcastExchange -Input [2]: [cd_demo_sk#52, cd_marital_status#53] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +(56) CometBroadcastExchange +Input [2]: [cd_demo_sk#47, cd_marital_status#48] +Arguments: [cd_demo_sk#47, cd_marital_status#48] -(62) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#52] -Join type: Inner -Join condition: None +(57) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] +Right output [2]: [cd_demo_sk#47, cd_marital_status#48] +Arguments: [ss_cdemo_sk#3], [cd_demo_sk#47], Inner, BuildRight -(63) Project [codegen id : 24] -Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_marital_status#53] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_demo_sk#52, cd_marital_status#53] +(58) CometProject +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_demo_sk#47, cd_marital_status#48] +Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_marital_status#48], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_marital_status#48] -(64) ReusedExchange [Reuses operator id: 61] -Output [2]: [cd_demo_sk#54, cd_marital_status#55] +(59) ReusedExchange [Reuses operator id: 56] +Output [2]: [cd_demo_sk#49, cd_marital_status#50] -(65) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [c_current_cdemo_sk#43] -Right keys [1]: [cd_demo_sk#54] -Join type: Inner -Join condition: NOT (cd_marital_status#53 = cd_marital_status#55) +(60) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_marital_status#48] +Right output [2]: [cd_demo_sk#49, cd_marital_status#50] +Arguments: [c_current_cdemo_sk#38], [cd_demo_sk#49], Inner, NOT (cd_marital_status#48 = cd_marital_status#50), BuildRight -(66) Project [codegen id : 24] -Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_marital_status#53, cd_demo_sk#54, cd_marital_status#55] +(61) CometProject +Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_marital_status#48, cd_demo_sk#49, cd_marital_status#50] +Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] -(67) Scan parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#56] +(62) Scan parquet spark_catalog.default.promotion +Output [1]: [p_promo_sk#51] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_promo_sk)] ReadSchema: struct -(68) CometFilter -Input [1]: [p_promo_sk#56] -Condition : isnotnull(p_promo_sk#56) +(63) CometFilter +Input [1]: [p_promo_sk#51] +Condition : isnotnull(p_promo_sk#51) -(69) ColumnarToRow [codegen id : 16] -Input [1]: [p_promo_sk#56] +(64) CometBroadcastExchange +Input [1]: [p_promo_sk#51] +Arguments: [p_promo_sk#51] -(70) BroadcastExchange -Input [1]: [p_promo_sk#56] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] +(65) CometBroadcastHashJoin +Left output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] +Right output [1]: [p_promo_sk#51] +Arguments: [ss_promo_sk#7], [p_promo_sk#51], Inner, BuildRight -(71) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_promo_sk#7] -Right keys [1]: [p_promo_sk#56] -Join type: Inner -Join condition: None - -(72) Project [codegen id : 24] -Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, p_promo_sk#56] +(66) CometProject +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, p_promo_sk#51] +Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] -(73) Scan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#57, hd_income_band_sk#58] +(67) Scan parquet spark_catalog.default.household_demographics +Output [2]: [hd_demo_sk#52, hd_income_band_sk#53] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] ReadSchema: struct -(74) CometFilter -Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] -Condition : (isnotnull(hd_demo_sk#57) AND isnotnull(hd_income_band_sk#58)) - -(75) ColumnarToRow [codegen id : 17] -Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] +(68) CometFilter +Input [2]: [hd_demo_sk#52, hd_income_band_sk#53] +Condition : (isnotnull(hd_demo_sk#52) AND isnotnull(hd_income_band_sk#53)) -(76) BroadcastExchange -Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] +(69) CometBroadcastExchange +Input [2]: [hd_demo_sk#52, hd_income_band_sk#53] +Arguments: [hd_demo_sk#52, hd_income_band_sk#53] -(77) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_hdemo_sk#4] -Right keys [1]: [hd_demo_sk#57] -Join type: Inner -Join condition: None +(70) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] +Right output [2]: [hd_demo_sk#52, hd_income_band_sk#53] +Arguments: [ss_hdemo_sk#4], [hd_demo_sk#52], Inner, BuildRight -(78) Project [codegen id : 24] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_demo_sk#57, hd_income_band_sk#58] +(71) CometProject +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_demo_sk#52, hd_income_band_sk#53] +Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53] -(79) ReusedExchange [Reuses operator id: 76] -Output [2]: [hd_demo_sk#59, hd_income_band_sk#60] +(72) ReusedExchange [Reuses operator id: 69] +Output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -(80) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [c_current_hdemo_sk#44] -Right keys [1]: [hd_demo_sk#59] -Join type: Inner -Join condition: None +(73) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53] +Right output [2]: [hd_demo_sk#54, hd_income_band_sk#55] +Arguments: [c_current_hdemo_sk#39], [hd_demo_sk#54], Inner, BuildRight -(81) Project [codegen id : 24] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60] -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_demo_sk#59, hd_income_band_sk#60] +(74) CometProject +Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_demo_sk#54, hd_income_band_sk#55] +Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55] -(82) Scan parquet spark_catalog.default.customer_address -Output [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +(75) Scan parquet spark_catalog.default.customer_address +Output [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(83) CometFilter -Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -Condition : isnotnull(ca_address_sk#61) +(76) CometFilter +Input [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +Condition : isnotnull(ca_address_sk#56) -(84) ColumnarToRow [codegen id : 19] -Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +(77) CometBroadcastExchange +Input [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +Arguments: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] -(85) BroadcastExchange -Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=11] +(78) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55] +Right output [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +Arguments: [ss_addr_sk#5], [ca_address_sk#56], Inner, BuildRight -(86) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_addr_sk#5] -Right keys [1]: [ca_address_sk#61] -Join type: Inner -Join condition: None - -(87) Project [codegen id : 24] -Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +(79) CometProject +Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] -(88) ReusedExchange [Reuses operator id: 85] -Output [5]: [ca_address_sk#66, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] +(80) ReusedExchange [Reuses operator id: 77] +Output [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -(89) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [c_current_addr_sk#45] -Right keys [1]: [ca_address_sk#66] -Join type: Inner -Join condition: None +(81) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +Right output [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Arguments: [c_current_addr_sk#40], [ca_address_sk#61], Inner, BuildRight -(90) Project [codegen id : 24] -Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_address_sk#66, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] +(82) CometProject +Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -(91) Scan parquet spark_catalog.default.income_band -Output [1]: [ib_income_band_sk#71] +(83) Scan parquet spark_catalog.default.income_band +Output [1]: [ib_income_band_sk#66] Batched: true Location [not included in comparison]/{warehouse_dir}/income_band] PushedFilters: [IsNotNull(ib_income_band_sk)] ReadSchema: struct -(92) CometFilter -Input [1]: [ib_income_band_sk#71] -Condition : isnotnull(ib_income_band_sk#71) +(84) CometFilter +Input [1]: [ib_income_band_sk#66] +Condition : isnotnull(ib_income_band_sk#66) -(93) ColumnarToRow [codegen id : 21] -Input [1]: [ib_income_band_sk#71] +(85) CometBroadcastExchange +Input [1]: [ib_income_band_sk#66] +Arguments: [ib_income_band_sk#66] -(94) BroadcastExchange -Input [1]: [ib_income_band_sk#71] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] - -(95) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [hd_income_band_sk#58] -Right keys [1]: [ib_income_band_sk#71] -Join type: Inner -Join condition: None +(86) CometBroadcastHashJoin +Left output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Right output [1]: [ib_income_band_sk#66] +Arguments: [hd_income_band_sk#53], [ib_income_band_sk#66], Inner, BuildRight -(96) Project [codegen id : 24] -Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, ib_income_band_sk#71] +(87) CometProject +Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ib_income_band_sk#66] +Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -(97) ReusedExchange [Reuses operator id: 94] -Output [1]: [ib_income_band_sk#72] +(88) ReusedExchange [Reuses operator id: 85] +Output [1]: [ib_income_band_sk#67] -(98) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [hd_income_band_sk#60] -Right keys [1]: [ib_income_band_sk#72] -Join type: Inner -Join condition: None +(89) CometBroadcastHashJoin +Left output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Right output [1]: [ib_income_band_sk#67] +Arguments: [hd_income_band_sk#55], [ib_income_band_sk#67], Inner, BuildRight -(99) Project [codegen id : 24] -Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, ib_income_band_sk#72] +(90) CometProject +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ib_income_band_sk#67] +Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -(100) Scan parquet spark_catalog.default.item -Output [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] +(91) Scan parquet spark_catalog.default.item +Output [4]: [i_item_sk#68, i_current_price#69, i_color#70, i_product_name#71] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), In(i_color, [burlywood ,floral ,indian ,medium ,purple ,spring ]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] ReadSchema: struct -(101) CometFilter -Input [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] -Condition : ((((((isnotnull(i_current_price#74) AND i_color#75 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#74 >= 64.00)) AND (i_current_price#74 <= 74.00)) AND (i_current_price#74 >= 65.00)) AND (i_current_price#74 <= 79.00)) AND isnotnull(i_item_sk#73)) - -(102) CometProject -Input [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] -Arguments: [i_item_sk#73, i_product_name#76], [i_item_sk#73, i_product_name#76] +(92) CometFilter +Input [4]: [i_item_sk#68, i_current_price#69, i_color#70, i_product_name#71] +Condition : ((((((isnotnull(i_current_price#69) AND i_color#70 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#69 >= 64.00)) AND (i_current_price#69 <= 74.00)) AND (i_current_price#69 >= 65.00)) AND (i_current_price#69 <= 79.00)) AND isnotnull(i_item_sk#68)) -(103) ColumnarToRow [codegen id : 23] -Input [2]: [i_item_sk#73, i_product_name#76] +(93) CometProject +Input [4]: [i_item_sk#68, i_current_price#69, i_color#70, i_product_name#71] +Arguments: [i_item_sk#68, i_product_name#71], [i_item_sk#68, i_product_name#71] -(104) BroadcastExchange -Input [2]: [i_item_sk#73, i_product_name#76] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] +(94) CometBroadcastExchange +Input [2]: [i_item_sk#68, i_product_name#71] +Arguments: [i_item_sk#68, i_product_name#71] -(105) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#73] -Join type: Inner -Join condition: None +(95) CometBroadcastHashJoin +Left output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Right output [2]: [i_item_sk#68, i_product_name#71] +Arguments: [ss_item_sk#1], [i_item_sk#68], Inner, BuildRight -(106) Project [codegen id : 24] -Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, d_year#49, d_year#51, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] +(96) CometProject +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, i_item_sk#68, i_product_name#71] +Arguments: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#44, d_year#46, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, i_item_sk#68, i_product_name#71], [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#44, d_year#46, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, i_item_sk#68, i_product_name#71] -(107) HashAggregate [codegen id : 24] -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, d_year#49, d_year#51, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51] +(97) CometHashAggregate +Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#44, d_year#46, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, i_item_sk#68, i_product_name#71] +Keys [15]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46] Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count#77, sum#78, sum#79, sum#80] -Results [19]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51, count#81, sum#82, sum#83, sum#84] -(108) HashAggregate [codegen id : 24] -Input [19]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51, count#81, sum#82, sum#83, sum#84] -Keys [15]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51] +(98) CometHashAggregate +Input [19]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46, count#72, sum#73, sum#74, sum#75] +Keys [15]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46] Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count(1)#85, sum(UnscaledValue(ss_wholesale_cost#9))#86, sum(UnscaledValue(ss_list_price#10))#87, sum(UnscaledValue(ss_coupon_amt#11))#88] -Results [17]: [i_product_name#76 AS product_name#89, i_item_sk#73 AS item_sk#90, s_store_name#40 AS store_name#91, s_zip#41 AS store_zip#92, ca_street_number#62 AS b_street_number#93, ca_street_name#63 AS b_streen_name#94, ca_city#64 AS b_city#95, ca_zip#65 AS b_zip#96, ca_street_number#67 AS c_street_number#97, ca_street_name#68 AS c_street_name#98, ca_city#69 AS c_city#99, ca_zip#70 AS c_zip#100, d_year#38 AS syear#101, count(1)#85 AS cnt#102, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#86,17,2) AS s1#103, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#87,17,2) AS s2#104, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#88,17,2) AS s3#105] -(109) Exchange -Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] -Arguments: hashpartitioning(item_sk#90, store_name#91, store_zip#92, 5), ENSURE_REQUIREMENTS, [plan_id=14] +(99) CometColumnarExchange +Input [17]: [product_name#76, item_sk#77, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92] +Arguments: hashpartitioning(item_sk#77, store_name#78, store_zip#79, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(100) CometSort +Input [17]: [product_name#76, item_sk#77, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92] +Arguments: [product_name#76, item_sk#77, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92], [item_sk#77 ASC NULLS FIRST, store_name#78 ASC NULLS FIRST, store_zip#79 ASC NULLS FIRST] -(110) Sort [codegen id : 25] -Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] -Arguments: [item_sk#90 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, store_zip#92 ASC NULLS FIRST], false, 0 +(101) ColumnarToRow [codegen id : 1] +Input [17]: [product_name#76, item_sk#77, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92] -(111) Scan parquet spark_catalog.default.store_sales -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] +(102) Scan parquet spark_catalog.default.store_sales +Output [12]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_ticket_number#100, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#117), dynamicpruningexpression(ss_sold_date_sk#117 IN dynamicpruning#118)] +PartitionFilters: [isnotnull(ss_sold_date_sk#104), dynamicpruningexpression(ss_sold_date_sk#104 IN dynamicpruning#105)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct -(112) CometFilter -Input [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] -Condition : (((((((isnotnull(ss_item_sk#106) AND isnotnull(ss_ticket_number#113)) AND isnotnull(ss_store_sk#111)) AND isnotnull(ss_customer_sk#107)) AND isnotnull(ss_cdemo_sk#108)) AND isnotnull(ss_promo_sk#112)) AND isnotnull(ss_hdemo_sk#109)) AND isnotnull(ss_addr_sk#110)) +(103) CometFilter +Input [12]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_ticket_number#100, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] +Condition : (((((((isnotnull(ss_item_sk#93) AND isnotnull(ss_ticket_number#100)) AND isnotnull(ss_store_sk#98)) AND isnotnull(ss_customer_sk#94)) AND isnotnull(ss_cdemo_sk#95)) AND isnotnull(ss_promo_sk#99)) AND isnotnull(ss_hdemo_sk#96)) AND isnotnull(ss_addr_sk#97)) -(113) CometBroadcastExchange -Input [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] -Arguments: [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] +(104) CometBroadcastExchange +Input [12]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_ticket_number#100, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] +Arguments: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_ticket_number#100, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] -(114) Scan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] +(105) Scan parquet spark_catalog.default.store_returns +Output [3]: [sr_item_sk#106, sr_ticket_number#107, sr_returned_date_sk#108] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] ReadSchema: struct -(115) CometFilter -Input [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] -Condition : (isnotnull(sr_item_sk#119) AND isnotnull(sr_ticket_number#120)) +(106) CometFilter +Input [3]: [sr_item_sk#106, sr_ticket_number#107, sr_returned_date_sk#108] +Condition : (isnotnull(sr_item_sk#106) AND isnotnull(sr_ticket_number#107)) -(116) CometProject -Input [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] -Arguments: [sr_item_sk#119, sr_ticket_number#120], [sr_item_sk#119, sr_ticket_number#120] +(107) CometProject +Input [3]: [sr_item_sk#106, sr_ticket_number#107, sr_returned_date_sk#108] +Arguments: [sr_item_sk#106, sr_ticket_number#107], [sr_item_sk#106, sr_ticket_number#107] -(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, BuildLeft +(108) CometBroadcastHashJoin +Left output [12]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_ticket_number#100, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] +Right output [2]: [sr_item_sk#106, sr_ticket_number#107] +Arguments: [ss_item_sk#93, ss_ticket_number#100], [sr_item_sk#106, sr_ticket_number#107], 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] -Arguments: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117], [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +(109) CometProject +Input [14]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_ticket_number#100, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104, sr_item_sk#106, sr_ticket_number#107] +Arguments: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104], [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] -(119) ColumnarToRow [codegen id : 26] -Input [11]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +(110) CometColumnarExchange +Input [11]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] +Arguments: hashpartitioning(ss_item_sk#93, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(120) Exchange -Input [11]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -Arguments: hashpartitioning(ss_item_sk#106, 5), ENSURE_REQUIREMENTS, [plan_id=15] +(111) CometSort +Input [11]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] +Arguments: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104], [ss_item_sk#93 ASC NULLS FIRST] -(121) Sort [codegen id : 27] -Input [11]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -Arguments: [ss_item_sk#106 ASC NULLS FIRST], false, 0 +(112) ReusedExchange [Reuses operator id: 24] +Output [4]: [cs_item_sk#109, sum#110, sum#111, isEmpty#112] -(122) ReusedExchange [Reuses operator id: 27] -Output [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] +(113) CometHashAggregate +Input [4]: [cs_item_sk#109, sum#110, sum#111, isEmpty#112] +Keys [1]: [cs_item_sk#109] +Functions [2]: [sum(UnscaledValue(cs_ext_list_price#113)), sum(((cr_refunded_cash#114 + cr_reversed_charge#115) + cr_store_credit#116))] -(123) HashAggregate [codegen id : 33] -Input [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] -Keys [1]: [cs_item_sk#122] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#126)), sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#126))#33, sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))#34] -Results [3]: [cs_item_sk#122, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#126))#33,17,2) AS sale#35, sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))#34 AS refund#36] +(114) CometFilter +Input [3]: [cs_item_sk#109, sale#30, refund#31] +Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) -(124) Filter [codegen id : 33] -Input [3]: [cs_item_sk#122, sale#35, refund#36] -Condition : ((isnotnull(sale#35) AND isnotnull(refund#36)) AND (cast(sale#35 as decimal(21,2)) > (2 * refund#36))) +(115) CometProject +Input [3]: [cs_item_sk#109, sale#30, refund#31] +Arguments: [cs_item_sk#109], [cs_item_sk#109] -(125) Project [codegen id : 33] -Output [1]: [cs_item_sk#122] -Input [3]: [cs_item_sk#122, sale#35, refund#36] +(116) CometSort +Input [1]: [cs_item_sk#109] +Arguments: [cs_item_sk#109], [cs_item_sk#109 ASC NULLS FIRST] -(126) Sort [codegen id : 33] -Input [1]: [cs_item_sk#122] -Arguments: [cs_item_sk#122 ASC NULLS FIRST], false, 0 +(117) CometSortMergeJoin +Left output [11]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] +Right output [1]: [cs_item_sk#109] +Arguments: [ss_item_sk#93], [cs_item_sk#109], Inner -(127) SortMergeJoin [codegen id : 49] -Left keys [1]: [ss_item_sk#106] -Right keys [1]: [cs_item_sk#122] -Join type: Inner -Join condition: None +(118) CometProject +Input [12]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104, cs_item_sk#109] +Arguments: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104], [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] -(128) Project [codegen id : 49] -Output [11]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -Input [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, cs_item_sk#122] +(119) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#117, d_year#118] +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 -(129) ReusedExchange [Reuses operator id: 189] -Output [2]: [d_date_sk#130, d_year#131] +(120) CometFilter +Input [2]: [d_date_sk#117, d_year#118] +Condition : ((isnotnull(d_year#118) AND (d_year#118 = 2000)) AND isnotnull(d_date_sk#117)) -(130) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [ss_sold_date_sk#117] -Right keys [1]: [d_date_sk#130] -Join type: Inner -Join condition: None +(121) CometBroadcastExchange +Input [2]: [d_date_sk#117, d_year#118] +Arguments: [d_date_sk#117, d_year#118] -(131) Project [codegen id : 49] -Output [11]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131] -Input [13]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, d_date_sk#130, d_year#131] +(122) CometBroadcastHashJoin +Left output [11]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] +Right output [2]: [d_date_sk#117, d_year#118] +Arguments: [ss_sold_date_sk#104], [d_date_sk#117], Inner, BuildRight -(132) ReusedExchange [Reuses operator id: 40] -Output [3]: [s_store_sk#132, s_store_name#133, s_zip#134] +(123) CometProject +Input [13]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104, d_date_sk#117, d_year#118] +Arguments: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118], [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118] -(133) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [ss_store_sk#111] -Right keys [1]: [s_store_sk#132] -Join type: Inner -Join condition: None +(124) ReusedExchange [Reuses operator id: 38] +Output [3]: [s_store_sk#119, s_store_name#120, s_zip#121] -(134) Project [codegen id : 49] -Output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134] -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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_sk#132, s_store_name#133, s_zip#134] +(125) CometBroadcastHashJoin +Left output [11]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118] +Right output [3]: [s_store_sk#119, s_store_name#120, s_zip#121] +Arguments: [ss_store_sk#98], [s_store_sk#119], Inner, BuildRight -(135) ReusedExchange [Reuses operator id: 46] -Output [6]: [c_customer_sk#135, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, c_first_sales_date_sk#140] +(126) CometProject +Input [14]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_sk#119, s_store_name#120, s_zip#121] +Arguments: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121], [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121] -(136) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [ss_customer_sk#107] -Right keys [1]: [c_customer_sk#135] -Join type: Inner -Join condition: None +(127) ReusedExchange [Reuses operator id: 43] +Output [6]: [c_customer_sk#122, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, c_first_sales_date_sk#127] -(137) Project [codegen id : 49] -Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, c_first_sales_date_sk#140] -Input [18]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_customer_sk#135, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, c_first_sales_date_sk#140] +(128) CometBroadcastHashJoin +Left output [12]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121] +Right output [6]: [c_customer_sk#122, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, c_first_sales_date_sk#127] +Arguments: [ss_customer_sk#94], [c_customer_sk#122], Inner, BuildRight -(138) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#141, d_year#142] +(129) CometProject +Input [18]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_customer_sk#122, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, c_first_sales_date_sk#127] +Arguments: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, c_first_sales_date_sk#127], [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, c_first_sales_date_sk#127] -(139) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [c_first_sales_date_sk#140] -Right keys [1]: [d_date_sk#141] -Join type: Inner -Join condition: None +(130) ReusedExchange [Reuses operator id: 48] +Output [2]: [d_date_sk#128, d_year#129] -(140) Project [codegen id : 49] -Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, d_year#142] -Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, c_first_sales_date_sk#140, d_date_sk#141, d_year#142] +(131) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, c_first_sales_date_sk#127] +Right output [2]: [d_date_sk#128, d_year#129] +Arguments: [c_first_sales_date_sk#127], [d_date_sk#128], Inner, BuildRight -(141) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#143, d_year#144] +(132) CometProject +Input [18]: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, c_first_sales_date_sk#127, d_date_sk#128, d_year#129] +Arguments: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, d_year#129], [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, d_year#129] -(142) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [c_first_shipto_date_sk#139] -Right keys [1]: [d_date_sk#143] -Join type: Inner -Join condition: None +(133) ReusedExchange [Reuses operator id: 48] +Output [2]: [d_date_sk#130, d_year#131] -(143) Project [codegen id : 49] -Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144] -Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, d_year#142, d_date_sk#143, d_year#144] +(134) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, d_year#129] +Right output [2]: [d_date_sk#130, d_year#131] +Arguments: [c_first_shipto_date_sk#126], [d_date_sk#130], Inner, BuildRight -(144) ReusedExchange [Reuses operator id: 61] -Output [2]: [cd_demo_sk#145, cd_marital_status#146] +(135) CometProject +Input [18]: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, d_year#129, d_date_sk#130, d_year#131] +Arguments: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131], [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131] -(145) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [ss_cdemo_sk#108] -Right keys [1]: [cd_demo_sk#145] -Join type: Inner -Join condition: None +(136) ReusedExchange [Reuses operator id: 56] +Output [2]: [cd_demo_sk#132, cd_marital_status#133] -(146) Project [codegen id : 49] -Output [16]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, cd_marital_status#146] -Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, cd_demo_sk#145, cd_marital_status#146] +(137) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131] +Right output [2]: [cd_demo_sk#132, cd_marital_status#133] +Arguments: [ss_cdemo_sk#95], [cd_demo_sk#132], Inner, BuildRight -(147) ReusedExchange [Reuses operator id: 61] -Output [2]: [cd_demo_sk#147, cd_marital_status#148] +(138) CometProject +Input [18]: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, cd_demo_sk#132, cd_marital_status#133] +Arguments: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, cd_marital_status#133], [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, cd_marital_status#133] -(148) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [c_current_cdemo_sk#136] -Right keys [1]: [cd_demo_sk#147] -Join type: Inner -Join condition: NOT (cd_marital_status#146 = cd_marital_status#148) +(139) ReusedExchange [Reuses operator id: 56] +Output [2]: [cd_demo_sk#134, cd_marital_status#135] -(149) Project [codegen id : 49] -Output [14]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144] -Input [18]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, cd_marital_status#146, cd_demo_sk#147, cd_marital_status#148] +(140) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, cd_marital_status#133] +Right output [2]: [cd_demo_sk#134, cd_marital_status#135] +Arguments: [c_current_cdemo_sk#123], [cd_demo_sk#134], Inner, NOT (cd_marital_status#133 = cd_marital_status#135), BuildRight -(150) ReusedExchange [Reuses operator id: 70] -Output [1]: [p_promo_sk#149] +(141) CometProject +Input [18]: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, cd_marital_status#133, cd_demo_sk#134, cd_marital_status#135] +Arguments: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131], [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131] -(151) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [ss_promo_sk#112] -Right keys [1]: [p_promo_sk#149] -Join type: Inner -Join condition: None +(142) ReusedExchange [Reuses operator id: 64] +Output [1]: [p_promo_sk#136] -(152) Project [codegen id : 49] -Output [13]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144] -Input [15]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, p_promo_sk#149] +(143) CometBroadcastHashJoin +Left output [14]: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131] +Right output [1]: [p_promo_sk#136] +Arguments: [ss_promo_sk#99], [p_promo_sk#136], Inner, BuildRight -(153) ReusedExchange [Reuses operator id: 76] -Output [2]: [hd_demo_sk#150, hd_income_band_sk#151] +(144) CometProject +Input [15]: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, p_promo_sk#136] +Arguments: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131], [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131] -(154) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [ss_hdemo_sk#109] -Right keys [1]: [hd_demo_sk#150] -Join type: Inner -Join condition: None +(145) ReusedExchange [Reuses operator id: 69] +Output [2]: [hd_demo_sk#137, hd_income_band_sk#138] -(155) Project [codegen id : 49] -Output [13]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151] -Input [15]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, hd_demo_sk#150, hd_income_band_sk#151] +(146) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131] +Right output [2]: [hd_demo_sk#137, hd_income_band_sk#138] +Arguments: [ss_hdemo_sk#96], [hd_demo_sk#137], Inner, BuildRight -(156) ReusedExchange [Reuses operator id: 76] -Output [2]: [hd_demo_sk#152, hd_income_band_sk#153] +(147) CometProject +Input [15]: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, hd_demo_sk#137, hd_income_band_sk#138] +Arguments: [ss_item_sk#93, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138], [ss_item_sk#93, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138] -(157) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [c_current_hdemo_sk#137] -Right keys [1]: [hd_demo_sk#152] -Join type: Inner -Join condition: None +(148) ReusedExchange [Reuses operator id: 69] +Output [2]: [hd_demo_sk#139, hd_income_band_sk#140] -(158) Project [codegen id : 49] -Output [13]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153] -Input [15]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_demo_sk#152, hd_income_band_sk#153] +(149) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#93, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138] +Right output [2]: [hd_demo_sk#139, hd_income_band_sk#140] +Arguments: [c_current_hdemo_sk#124], [hd_demo_sk#139], Inner, BuildRight -(159) ReusedExchange [Reuses operator id: 85] -Output [5]: [ca_address_sk#154, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] +(150) CometProject +Input [15]: [ss_item_sk#93, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_demo_sk#139, hd_income_band_sk#140] +Arguments: [ss_item_sk#93, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140], [ss_item_sk#93, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140] -(160) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [ss_addr_sk#110] -Right keys [1]: [ca_address_sk#154] -Join type: Inner -Join condition: None +(151) ReusedExchange [Reuses operator id: 77] +Output [5]: [ca_address_sk#141, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] -(161) Project [codegen id : 49] -Output [16]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] -Input [18]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_address_sk#154, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] +(152) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#93, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140] +Right output [5]: [ca_address_sk#141, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Arguments: [ss_addr_sk#97], [ca_address_sk#141], Inner, BuildRight -(162) ReusedExchange [Reuses operator id: 85] -Output [5]: [ca_address_sk#159, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] +(153) CometProject +Input [18]: [ss_item_sk#93, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_address_sk#141, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Arguments: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145], [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] -(163) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [c_current_addr_sk#138] -Right keys [1]: [ca_address_sk#159] -Join type: Inner -Join condition: None +(154) ReusedExchange [Reuses operator id: 77] +Output [5]: [ca_address_sk#146, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] -(164) Project [codegen id : 49] -Output [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] -Input [21]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_address_sk#159, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] +(155) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Right output [5]: [ca_address_sk#146, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] +Arguments: [c_current_addr_sk#125], [ca_address_sk#146], Inner, BuildRight -(165) ReusedExchange [Reuses operator id: 94] -Output [1]: [ib_income_band_sk#164] +(156) CometProject +Input [21]: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_address_sk#146, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] +Arguments: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150], [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] -(166) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [hd_income_band_sk#151] -Right keys [1]: [ib_income_band_sk#164] -Join type: Inner -Join condition: None +(157) ReusedExchange [Reuses operator id: 85] +Output [1]: [ib_income_band_sk#151] -(167) Project [codegen id : 49] -Output [18]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] -Input [20]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, ib_income_band_sk#164] +(158) CometBroadcastHashJoin +Left output [19]: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] +Right output [1]: [ib_income_band_sk#151] +Arguments: [hd_income_band_sk#138], [ib_income_band_sk#151], Inner, BuildRight -(168) ReusedExchange [Reuses operator id: 94] -Output [1]: [ib_income_band_sk#165] +(159) CometProject +Input [20]: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, ib_income_band_sk#151] +Arguments: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150], [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] -(169) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [hd_income_band_sk#153] -Right keys [1]: [ib_income_band_sk#165] -Join type: Inner -Join condition: None +(160) ReusedExchange [Reuses operator id: 85] +Output [1]: [ib_income_band_sk#152] -(170) Project [codegen id : 49] -Output [17]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] -Input [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, ib_income_band_sk#165] +(161) CometBroadcastHashJoin +Left output [18]: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] +Right output [1]: [ib_income_band_sk#152] +Arguments: [hd_income_band_sk#140], [ib_income_band_sk#152], Inner, BuildRight -(171) ReusedExchange [Reuses operator id: 104] -Output [2]: [i_item_sk#166, i_product_name#167] +(162) CometProject +Input [19]: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, ib_income_band_sk#152] +Arguments: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150], [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] -(172) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [ss_item_sk#106] -Right keys [1]: [i_item_sk#166] -Join type: Inner -Join condition: None - -(173) Project [codegen id : 49] -Output [18]: [ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, d_year#142, d_year#144, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, i_item_sk#166, i_product_name#167] -Input [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, i_item_sk#166, i_product_name#167] - -(174) HashAggregate [codegen id : 49] -Input [18]: [ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, d_year#142, d_year#144, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, i_item_sk#166, i_product_name#167] -Keys [15]: [i_product_name#167, i_item_sk#166, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, d_year#131, d_year#142, d_year#144] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#114)), partial_sum(UnscaledValue(ss_list_price#115)), partial_sum(UnscaledValue(ss_coupon_amt#116))] -Aggregate Attributes [4]: [count#77, sum#168, sum#169, sum#170] -Results [19]: [i_product_name#167, i_item_sk#166, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, d_year#131, d_year#142, d_year#144, count#81, sum#171, sum#172, sum#173] - -(175) HashAggregate [codegen id : 49] -Input [19]: [i_product_name#167, i_item_sk#166, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, d_year#131, d_year#142, d_year#144, count#81, sum#171, sum#172, sum#173] -Keys [15]: [i_product_name#167, i_item_sk#166, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, d_year#131, d_year#142, d_year#144] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#114)), sum(UnscaledValue(ss_list_price#115)), sum(UnscaledValue(ss_coupon_amt#116))] -Aggregate Attributes [4]: [count(1)#85, sum(UnscaledValue(ss_wholesale_cost#114))#86, sum(UnscaledValue(ss_list_price#115))#87, sum(UnscaledValue(ss_coupon_amt#116))#88] -Results [8]: [i_item_sk#166 AS item_sk#174, s_store_name#133 AS store_name#175, s_zip#134 AS store_zip#176, d_year#131 AS syear#177, count(1)#85 AS cnt#178, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#114))#86,17,2) AS s1#179, MakeDecimal(sum(UnscaledValue(ss_list_price#115))#87,17,2) AS s2#180, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#116))#88,17,2) AS s3#181] - -(176) Exchange -Input [8]: [item_sk#174, store_name#175, store_zip#176, syear#177, cnt#178, s1#179, s2#180, s3#181] -Arguments: hashpartitioning(item_sk#174, store_name#175, store_zip#176, 5), ENSURE_REQUIREMENTS, [plan_id=16] - -(177) Sort [codegen id : 50] -Input [8]: [item_sk#174, store_name#175, store_zip#176, syear#177, cnt#178, s1#179, s2#180, s3#181] -Arguments: [item_sk#174 ASC NULLS FIRST, store_name#175 ASC NULLS FIRST, store_zip#176 ASC NULLS FIRST], false, 0 - -(178) SortMergeJoin [codegen id : 51] -Left keys [3]: [item_sk#90, store_name#91, store_zip#92] -Right keys [3]: [item_sk#174, store_name#175, store_zip#176] +(163) ReusedExchange [Reuses operator id: 94] +Output [2]: [i_item_sk#153, i_product_name#154] + +(164) CometBroadcastHashJoin +Left output [17]: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] +Right output [2]: [i_item_sk#153, i_product_name#154] +Arguments: [ss_item_sk#93], [i_item_sk#153], Inner, BuildRight + +(165) CometProject +Input [19]: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, i_item_sk#153, i_product_name#154] +Arguments: [ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, d_year#129, d_year#131, s_store_name#120, s_zip#121, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, i_item_sk#153, i_product_name#154], [ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, d_year#129, d_year#131, s_store_name#120, s_zip#121, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, i_item_sk#153, i_product_name#154] + +(166) CometHashAggregate +Input [18]: [ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, d_year#129, d_year#131, s_store_name#120, s_zip#121, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, i_item_sk#153, i_product_name#154] +Keys [15]: [i_product_name#154, i_item_sk#153, s_store_name#120, s_zip#121, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, d_year#118, d_year#129, d_year#131] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#101)), partial_sum(UnscaledValue(ss_list_price#102)), partial_sum(UnscaledValue(ss_coupon_amt#103))] + +(167) CometHashAggregate +Input [19]: [i_product_name#154, i_item_sk#153, s_store_name#120, s_zip#121, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, d_year#118, d_year#129, d_year#131, count#72, sum#155, sum#156, sum#157] +Keys [15]: [i_product_name#154, i_item_sk#153, s_store_name#120, s_zip#121, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, d_year#118, d_year#129, d_year#131] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#101)), sum(UnscaledValue(ss_list_price#102)), sum(UnscaledValue(ss_coupon_amt#103))] + +(168) CometColumnarExchange +Input [8]: [item_sk#158, store_name#159, store_zip#160, syear#161, cnt#162, s1#163, s2#164, s3#165] +Arguments: hashpartitioning(item_sk#158, store_name#159, store_zip#160, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(169) CometSort +Input [8]: [item_sk#158, store_name#159, store_zip#160, syear#161, cnt#162, s1#163, s2#164, s3#165] +Arguments: [item_sk#158, store_name#159, store_zip#160, syear#161, cnt#162, s1#163, s2#164, s3#165], [item_sk#158 ASC NULLS FIRST, store_name#159 ASC NULLS FIRST, store_zip#160 ASC NULLS FIRST] + +(170) ColumnarToRow [codegen id : 2] +Input [8]: [item_sk#158, store_name#159, store_zip#160, syear#161, cnt#162, s1#163, s2#164, s3#165] + +(171) SortMergeJoin [codegen id : 3] +Left keys [3]: [item_sk#77, store_name#78, store_zip#79] +Right keys [3]: [item_sk#158, store_name#159, store_zip#160] Join type: Inner -Join condition: (cnt#178 <= cnt#102) +Join condition: (cnt#162 <= cnt#89) -(179) Project [codegen id : 51] -Output [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#179, s2#180, s3#181, syear#177, cnt#178] -Input [25]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, item_sk#174, store_name#175, store_zip#176, syear#177, cnt#178, s1#179, s2#180, s3#181] +(172) Project [codegen id : 3] +Output [21]: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162] +Input [25]: [product_name#76, item_sk#77, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, item_sk#158, store_name#159, store_zip#160, syear#161, cnt#162, s1#163, s2#164, s3#165] -(180) Exchange -Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#179, s2#180, s3#181, syear#177, cnt#178] -Arguments: rangepartitioning(product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#178 ASC NULLS FIRST, s1#103 ASC NULLS FIRST, s1#179 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=17] +(173) RowToColumnar +Input [21]: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162] -(181) Sort [codegen id : 52] -Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#179, s2#180, s3#181, syear#177, cnt#178] -Arguments: [product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#178 ASC NULLS FIRST, s1#103 ASC NULLS FIRST, s1#179 ASC NULLS FIRST], true, 0 +(174) CometColumnarExchange +Input [21]: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162] +Arguments: rangepartitioning(product_name#76 ASC NULLS FIRST, store_name#78 ASC NULLS FIRST, cnt#162 ASC NULLS FIRST, s1#90 ASC NULLS FIRST, s1#163 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(175) CometSort +Input [21]: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162] +Arguments: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162], [product_name#76 ASC NULLS FIRST, store_name#78 ASC NULLS FIRST, cnt#162 ASC NULLS FIRST, s1#90 ASC NULLS FIRST, s1#163 ASC NULLS FIRST] + +(176) ColumnarToRow [codegen id : 4] +Input [21]: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (185) -+- * ColumnarToRow (184) - +- CometFilter (183) - +- CometScan parquet spark_catalog.default.date_dim (182) +BroadcastExchange (180) ++- * ColumnarToRow (179) + +- CometFilter (178) + +- CometScan parquet spark_catalog.default.date_dim (177) -(182) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#37, d_year#38] +(177) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#32, d_year#33] 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 -(183) CometFilter -Input [2]: [d_date_sk#37, d_year#38] -Condition : ((isnotnull(d_year#38) AND (d_year#38 = 1999)) AND isnotnull(d_date_sk#37)) +(178) CometFilter +Input [2]: [d_date_sk#32, d_year#33] +Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(184) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#37, d_year#38] +(179) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#32, d_year#33] -(185) BroadcastExchange -Input [2]: [d_date_sk#37, d_year#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] +(180) BroadcastExchange +Input [2]: [d_date_sk#32, d_year#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -Subquery:2 Hosting operator id = 111 Hosting Expression = ss_sold_date_sk#117 IN dynamicpruning#118 -BroadcastExchange (189) -+- * ColumnarToRow (188) - +- CometFilter (187) - +- CometScan parquet spark_catalog.default.date_dim (186) +Subquery:2 Hosting operator id = 102 Hosting Expression = ss_sold_date_sk#104 IN dynamicpruning#105 +BroadcastExchange (184) ++- * ColumnarToRow (183) + +- CometFilter (182) + +- CometScan parquet spark_catalog.default.date_dim (181) -(186) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#130, d_year#131] +(181) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#117, d_year#118] 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 -(187) CometFilter -Input [2]: [d_date_sk#130, d_year#131] -Condition : ((isnotnull(d_year#131) AND (d_year#131 = 2000)) AND isnotnull(d_date_sk#130)) +(182) CometFilter +Input [2]: [d_date_sk#117, d_year#118] +Condition : ((isnotnull(d_year#118) AND (d_year#118 = 2000)) AND isnotnull(d_date_sk#117)) -(188) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#130, d_year#131] +(183) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#117, d_year#118] -(189) BroadcastExchange -Input [2]: [d_date_sk#130, d_year#131] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=19] +(184) BroadcastExchange +Input [2]: [d_date_sk#117, d_year#118] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt index 512a74f9f..70d0e66f8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt @@ -1,281 +1,199 @@ -WholeStageCodegen (52) - Sort [product_name,store_name,cnt,s1,s1] +WholeStageCodegen (4) + ColumnarToRow InputAdapter - Exchange [product_name,store_name,cnt,s1,s1] #1 - WholeStageCodegen (51) - Project [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - SortMergeJoin [item_sk,store_name,store_zip,item_sk,store_name,store_zip,cnt,cnt] - InputAdapter - WholeStageCodegen (25) - Sort [item_sk,store_name,store_zip] - InputAdapter - Exchange [item_sk,store_name,store_zip] #2 - WholeStageCodegen (24) - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] - Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SortMergeJoin [ss_item_sk,cs_item_sk] - InputAdapter - WholeStageCodegen (2) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #4 - CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - WholeStageCodegen (8) - Sort [cs_item_sk] - Project [cs_item_sk] - Filter [sale,refund] - HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] - InputAdapter - Exchange [cs_item_sk] #6 - WholeStageCodegen (7) - HashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [sum,sum,isEmpty,sum,sum,isEmpty] - Project [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] - InputAdapter - WholeStageCodegen (4) - Sort [cs_item_sk,cs_order_number] - InputAdapter - Exchange [cs_item_sk,cs_order_number] #7 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - InputAdapter - WholeStageCodegen (6) - Sort [cr_item_sk,cr_order_number] - InputAdapter - Exchange [cr_item_sk,cr_order_number] #8 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_name,s_zip] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (12) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #11 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (14) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk,cd_marital_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometFilter [p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk] - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (17) - 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 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (19) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 - InputAdapter - BroadcastExchange #16 - WholeStageCodegen (21) - ColumnarToRow - InputAdapter - CometFilter [ib_income_band_sk] - CometScan parquet spark_catalog.default.income_band [ib_income_band_sk] - InputAdapter - ReusedExchange [ib_income_band_sk] #16 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (23) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] - InputAdapter - WholeStageCodegen (50) - Sort [item_sk,store_name,store_zip] - InputAdapter - Exchange [item_sk,store_name,store_zip] #18 - WholeStageCodegen (49) - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] - Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SortMergeJoin [ss_item_sk,cs_item_sk] - InputAdapter - WholeStageCodegen (27) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #19 - WholeStageCodegen (26) - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #20 - CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #21 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - WholeStageCodegen (33) - Sort [cs_item_sk] - Project [cs_item_sk] - Filter [sale,refund] - HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] - InputAdapter - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #21 - InputAdapter - ReusedExchange [s_store_sk,s_store_name,s_zip] #9 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #10 - InputAdapter - ReusedExchange [d_date_sk,d_year] #11 - InputAdapter - ReusedExchange [d_date_sk,d_year] #11 - InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 - InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 - InputAdapter - ReusedExchange [p_promo_sk] #13 - InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 - InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 - InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 - InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 - InputAdapter - ReusedExchange [ib_income_band_sk] #16 - InputAdapter - ReusedExchange [ib_income_band_sk] #16 - InputAdapter - ReusedExchange [i_item_sk,i_product_name] #17 + CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] + CometColumnarExchange [product_name,store_name,cnt,s1,s1] #1 + RowToColumnar + WholeStageCodegen (3) + Project [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] + SortMergeJoin [item_sk,store_name,store_zip,item_sk,store_name,store_zip,cnt,cnt] + InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometSort [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3] + CometColumnarExchange [item_sk,store_name,store_zip] #2 + CometHashAggregate [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] + CometHashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum,ss_wholesale_cost,ss_list_price,ss_coupon_amt] + CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] + CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk] #3 + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #4 + CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [cs_item_sk,sale,refund] + CometHashAggregate [cs_item_sk,sale,refund,sum,sum,isEmpty,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] + CometColumnarExchange [cs_item_sk] #6 + CometHashAggregate [cs_item_sk,sum,sum,isEmpty,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometSortMergeJoin [cs_item_sk,cs_order_number,cs_ext_list_price,cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometSort [cs_item_sk,cs_order_number,cs_ext_list_price] + CometColumnarExchange [cs_item_sk,cs_order_number] #7 + CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometColumnarExchange [cr_item_sk,cr_order_number] #8 + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 + CometFilter [s_store_sk,s_store_name,s_zip] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #12 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk,d_year] #12 + CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 + CometFilter [cd_demo_sk,cd_marital_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + ReusedExchange [cd_demo_sk,cd_marital_status] #13 + CometBroadcastExchange [p_promo_sk] #14 + CometFilter [p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk] + CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 + CometFilter [hd_demo_sk,hd_income_band_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] + ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 + CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + CometBroadcastExchange [ib_income_band_sk] #17 + CometFilter [ib_income_band_sk] + CometScan parquet spark_catalog.default.income_band [ib_income_band_sk] + ReusedExchange [ib_income_band_sk] #17 + CometBroadcastExchange [i_item_sk,i_product_name] #18 + CometProject [i_item_sk,i_product_name] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] + InputAdapter + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometSort [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] + CometColumnarExchange [item_sk,store_name,store_zip] #19 + CometHashAggregate [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] + CometHashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum,ss_wholesale_cost,ss_list_price,ss_coupon_amt] + CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] + CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk] #20 + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #21 + CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #22 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [cs_item_sk,sale,refund] + CometHashAggregate [cs_item_sk,sale,refund,sum,sum,isEmpty,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] + ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 + CometBroadcastExchange [d_date_sk,d_year] #23 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [s_store_sk,s_store_name,s_zip] #10 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 + ReusedExchange [d_date_sk,d_year] #12 + ReusedExchange [d_date_sk,d_year] #12 + ReusedExchange [cd_demo_sk,cd_marital_status] #13 + ReusedExchange [cd_demo_sk,cd_marital_status] #13 + ReusedExchange [p_promo_sk] #14 + ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 + ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + ReusedExchange [ib_income_band_sk] #17 + ReusedExchange [ib_income_band_sk] #17 + ReusedExchange [i_item_sk,i_product_name] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/explain.txt index 6fd98e47b..f21890665 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/explain.txt @@ -1,75 +1,77 @@ == Physical Plan == -TakeOrderedAndProject (71) -+- * Filter (70) - +- Window (69) - +- WindowGroupLimit (68) - +- * Sort (67) - +- Exchange (66) - +- WindowGroupLimit (65) - +- * Sort (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) +TakeOrderedAndProject (73) ++- * Filter (72) + +- Window (71) + +- WindowGroupLimit (70) + +- * ColumnarToRow (69) + +- CometSort (68) + +- CometColumnarExchange (67) + +- RowToColumnar (66) + +- WindowGroupLimit (65) + +- * ColumnarToRow (64) + +- CometSort (63) + +- CometUnion (62) + :- CometHashAggregate (21) + : +- CometColumnarExchange (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) + :- CometHashAggregate (26) + : +- CometColumnarExchange (25) + : +- CometHashAggregate (24) + : +- CometHashAggregate (23) + : +- ReusedExchange (22) + :- CometHashAggregate (31) + : +- CometColumnarExchange (30) + : +- CometHashAggregate (29) + : +- CometHashAggregate (28) + : +- ReusedExchange (27) + :- CometHashAggregate (36) + : +- CometColumnarExchange (35) + : +- CometHashAggregate (34) + : +- CometHashAggregate (33) + : +- ReusedExchange (32) + :- CometHashAggregate (41) + : +- CometColumnarExchange (40) + : +- CometHashAggregate (39) + : +- CometHashAggregate (38) + : +- ReusedExchange (37) + :- CometHashAggregate (46) + : +- CometColumnarExchange (45) + : +- CometHashAggregate (44) + : +- CometHashAggregate (43) + : +- ReusedExchange (42) + :- CometHashAggregate (51) + : +- CometColumnarExchange (50) + : +- CometHashAggregate (49) + : +- CometHashAggregate (48) + : +- ReusedExchange (47) + :- CometHashAggregate (56) + : +- CometColumnarExchange (55) + : +- CometHashAggregate (54) + : +- CometHashAggregate (53) + : +- ReusedExchange (52) + +- CometHashAggregate (61) + +- CometColumnarExchange (60) + +- CometHashAggregate (59) + +- CometHashAggregate (58) + +- ReusedExchange (57) (1) Scan parquet spark_catalog.default.store_sales @@ -165,307 +167,272 @@ Input [10]: [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_st 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))] -(20) ColumnarToRow [codegen id : 1] +(20) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(22) HashAggregate [codegen id : 2] +(21) CometHashAggregate 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))#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] - -(23) ReusedExchange [Reuses operator id: 21] -Output [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, sum#31, isEmpty#32] - -(24) HashAggregate [codegen id : 4] -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, sum#31, isEmpty#32] -Keys [8]: [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] -Functions [1]: [sum(coalesce((ss_sales_price#33 * cast(ss_quantity#34 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#33 * cast(ss_quantity#34 as decimal(10,0))), 0.00))#21] -Results [8]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, sum(coalesce((ss_sales_price#33 * cast(ss_quantity#34 as decimal(10,0))), 0.00))#21 AS sumsales#35] - -(25) HashAggregate [codegen id : 4] -Input [8]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, sumsales#35] -Keys [7]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29] -Functions [1]: [partial_sum(sumsales#35)] -Aggregate Attributes [2]: [sum#36, isEmpty#37] -Results [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, sum#38, isEmpty#39] - -(26) 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, sum#38, isEmpty#39] -Arguments: hashpartitioning(i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, 5), ENSURE_REQUIREMENTS, [plan_id=2] - -(27) HashAggregate [codegen id : 5] -Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, sum#38, isEmpty#39] -Keys [7]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29] -Functions [1]: [sum(sumsales#35)] -Aggregate Attributes [1]: [sum(sumsales#35)#40] -Results [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, null AS s_store_id#41, sum(sumsales#35)#40 AS sumsales#42] - -(28) ReusedExchange [Reuses operator id: 21] -Output [10]: [i_category#43, i_class#44, i_brand#45, i_product_name#46, d_year#47, d_qoy#48, d_moy#49, s_store_id#50, sum#51, isEmpty#52] - -(29) HashAggregate [codegen id : 7] -Input [10]: [i_category#43, i_class#44, i_brand#45, i_product_name#46, d_year#47, d_qoy#48, d_moy#49, s_store_id#50, sum#51, isEmpty#52] -Keys [8]: [i_category#43, i_class#44, i_brand#45, i_product_name#46, d_year#47, d_qoy#48, d_moy#49, s_store_id#50] -Functions [1]: [sum(coalesce((ss_sales_price#53 * cast(ss_quantity#54 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#53 * cast(ss_quantity#54 as decimal(10,0))), 0.00))#21] -Results [7]: [i_category#43, i_class#44, i_brand#45, i_product_name#46, d_year#47, d_qoy#48, sum(coalesce((ss_sales_price#53 * cast(ss_quantity#54 as decimal(10,0))), 0.00))#21 AS sumsales#55] - -(30) HashAggregate [codegen id : 7] -Input [7]: [i_category#43, i_class#44, i_brand#45, i_product_name#46, d_year#47, d_qoy#48, sumsales#55] -Keys [6]: [i_category#43, i_class#44, i_brand#45, i_product_name#46, d_year#47, d_qoy#48] -Functions [1]: [partial_sum(sumsales#55)] -Aggregate Attributes [2]: [sum#56, isEmpty#57] -Results [8]: [i_category#43, i_class#44, i_brand#45, i_product_name#46, d_year#47, d_qoy#48, sum#58, isEmpty#59] - -(31) Exchange -Input [8]: [i_category#43, i_class#44, i_brand#45, i_product_name#46, d_year#47, d_qoy#48, sum#58, isEmpty#59] -Arguments: hashpartitioning(i_category#43, i_class#44, i_brand#45, i_product_name#46, d_year#47, d_qoy#48, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(32) HashAggregate [codegen id : 8] -Input [8]: [i_category#43, i_class#44, i_brand#45, i_product_name#46, d_year#47, d_qoy#48, sum#58, isEmpty#59] -Keys [6]: [i_category#43, i_class#44, i_brand#45, i_product_name#46, d_year#47, d_qoy#48] -Functions [1]: [sum(sumsales#55)] -Aggregate Attributes [1]: [sum(sumsales#55)#60] -Results [9]: [i_category#43, i_class#44, i_brand#45, i_product_name#46, d_year#47, d_qoy#48, null AS d_moy#61, null AS s_store_id#62, sum(sumsales#55)#60 AS sumsales#63] - -(33) ReusedExchange [Reuses operator id: 21] -Output [10]: [i_category#64, i_class#65, i_brand#66, i_product_name#67, d_year#68, d_qoy#69, d_moy#70, s_store_id#71, sum#72, isEmpty#73] - -(34) HashAggregate [codegen id : 10] -Input [10]: [i_category#64, i_class#65, i_brand#66, i_product_name#67, d_year#68, d_qoy#69, d_moy#70, s_store_id#71, sum#72, isEmpty#73] -Keys [8]: [i_category#64, i_class#65, i_brand#66, i_product_name#67, d_year#68, d_qoy#69, d_moy#70, s_store_id#71] -Functions [1]: [sum(coalesce((ss_sales_price#74 * cast(ss_quantity#75 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#74 * cast(ss_quantity#75 as decimal(10,0))), 0.00))#21] -Results [6]: [i_category#64, i_class#65, i_brand#66, i_product_name#67, d_year#68, sum(coalesce((ss_sales_price#74 * cast(ss_quantity#75 as decimal(10,0))), 0.00))#21 AS sumsales#76] - -(35) HashAggregate [codegen id : 10] -Input [6]: [i_category#64, i_class#65, i_brand#66, i_product_name#67, d_year#68, sumsales#76] -Keys [5]: [i_category#64, i_class#65, i_brand#66, i_product_name#67, d_year#68] -Functions [1]: [partial_sum(sumsales#76)] -Aggregate Attributes [2]: [sum#77, isEmpty#78] -Results [7]: [i_category#64, i_class#65, i_brand#66, i_product_name#67, d_year#68, sum#79, isEmpty#80] - -(36) Exchange -Input [7]: [i_category#64, i_class#65, i_brand#66, i_product_name#67, d_year#68, sum#79, isEmpty#80] -Arguments: hashpartitioning(i_category#64, i_class#65, i_brand#66, i_product_name#67, d_year#68, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(37) HashAggregate [codegen id : 11] -Input [7]: [i_category#64, i_class#65, i_brand#66, i_product_name#67, d_year#68, sum#79, isEmpty#80] -Keys [5]: [i_category#64, i_class#65, i_brand#66, i_product_name#67, d_year#68] -Functions [1]: [sum(sumsales#76)] -Aggregate Attributes [1]: [sum(sumsales#76)#81] -Results [9]: [i_category#64, i_class#65, i_brand#66, i_product_name#67, d_year#68, null AS d_qoy#82, null AS d_moy#83, null AS s_store_id#84, sum(sumsales#76)#81 AS sumsales#85] - -(38) ReusedExchange [Reuses operator id: 21] -Output [10]: [i_category#86, i_class#87, i_brand#88, i_product_name#89, d_year#90, d_qoy#91, d_moy#92, s_store_id#93, sum#94, isEmpty#95] - -(39) HashAggregate [codegen id : 13] -Input [10]: [i_category#86, i_class#87, i_brand#88, i_product_name#89, d_year#90, d_qoy#91, d_moy#92, s_store_id#93, sum#94, isEmpty#95] -Keys [8]: [i_category#86, i_class#87, i_brand#88, i_product_name#89, d_year#90, d_qoy#91, d_moy#92, s_store_id#93] -Functions [1]: [sum(coalesce((ss_sales_price#96 * cast(ss_quantity#97 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#96 * cast(ss_quantity#97 as decimal(10,0))), 0.00))#21] -Results [5]: [i_category#86, i_class#87, i_brand#88, i_product_name#89, sum(coalesce((ss_sales_price#96 * cast(ss_quantity#97 as decimal(10,0))), 0.00))#21 AS sumsales#98] - -(40) HashAggregate [codegen id : 13] -Input [5]: [i_category#86, i_class#87, i_brand#88, i_product_name#89, sumsales#98] -Keys [4]: [i_category#86, i_class#87, i_brand#88, i_product_name#89] -Functions [1]: [partial_sum(sumsales#98)] -Aggregate Attributes [2]: [sum#99, isEmpty#100] -Results [6]: [i_category#86, i_class#87, i_brand#88, i_product_name#89, sum#101, isEmpty#102] - -(41) Exchange -Input [6]: [i_category#86, i_class#87, i_brand#88, i_product_name#89, sum#101, isEmpty#102] -Arguments: hashpartitioning(i_category#86, i_class#87, i_brand#88, i_product_name#89, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(42) HashAggregate [codegen id : 14] -Input [6]: [i_category#86, i_class#87, i_brand#88, i_product_name#89, sum#101, isEmpty#102] -Keys [4]: [i_category#86, i_class#87, i_brand#88, i_product_name#89] -Functions [1]: [sum(sumsales#98)] -Aggregate Attributes [1]: [sum(sumsales#98)#103] -Results [9]: [i_category#86, i_class#87, i_brand#88, i_product_name#89, null AS d_year#104, null AS d_qoy#105, null AS d_moy#106, null AS s_store_id#107, sum(sumsales#98)#103 AS sumsales#108] - -(43) ReusedExchange [Reuses operator id: 21] -Output [10]: [i_category#109, i_class#110, i_brand#111, i_product_name#112, d_year#113, d_qoy#114, d_moy#115, s_store_id#116, sum#117, isEmpty#118] - -(44) HashAggregate [codegen id : 16] -Input [10]: [i_category#109, i_class#110, i_brand#111, i_product_name#112, d_year#113, d_qoy#114, d_moy#115, s_store_id#116, sum#117, isEmpty#118] -Keys [8]: [i_category#109, i_class#110, i_brand#111, i_product_name#112, d_year#113, d_qoy#114, d_moy#115, s_store_id#116] -Functions [1]: [sum(coalesce((ss_sales_price#119 * cast(ss_quantity#120 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#119 * cast(ss_quantity#120 as decimal(10,0))), 0.00))#21] -Results [4]: [i_category#109, i_class#110, i_brand#111, sum(coalesce((ss_sales_price#119 * cast(ss_quantity#120 as decimal(10,0))), 0.00))#21 AS sumsales#121] - -(45) HashAggregate [codegen id : 16] -Input [4]: [i_category#109, i_class#110, i_brand#111, sumsales#121] -Keys [3]: [i_category#109, i_class#110, i_brand#111] -Functions [1]: [partial_sum(sumsales#121)] -Aggregate Attributes [2]: [sum#122, isEmpty#123] -Results [5]: [i_category#109, i_class#110, i_brand#111, sum#124, isEmpty#125] - -(46) Exchange -Input [5]: [i_category#109, i_class#110, i_brand#111, sum#124, isEmpty#125] -Arguments: hashpartitioning(i_category#109, i_class#110, i_brand#111, 5), ENSURE_REQUIREMENTS, [plan_id=6] - -(47) HashAggregate [codegen id : 17] -Input [5]: [i_category#109, i_class#110, i_brand#111, sum#124, isEmpty#125] -Keys [3]: [i_category#109, i_class#110, i_brand#111] -Functions [1]: [sum(sumsales#121)] -Aggregate Attributes [1]: [sum(sumsales#121)#126] -Results [9]: [i_category#109, i_class#110, i_brand#111, null AS i_product_name#127, null AS d_year#128, null AS d_qoy#129, null AS d_moy#130, null AS s_store_id#131, sum(sumsales#121)#126 AS sumsales#132] - -(48) ReusedExchange [Reuses operator id: 21] -Output [10]: [i_category#133, i_class#134, i_brand#135, i_product_name#136, d_year#137, d_qoy#138, d_moy#139, s_store_id#140, sum#141, isEmpty#142] - -(49) HashAggregate [codegen id : 19] -Input [10]: [i_category#133, i_class#134, i_brand#135, i_product_name#136, d_year#137, d_qoy#138, d_moy#139, s_store_id#140, sum#141, isEmpty#142] -Keys [8]: [i_category#133, i_class#134, i_brand#135, i_product_name#136, d_year#137, d_qoy#138, d_moy#139, s_store_id#140] -Functions [1]: [sum(coalesce((ss_sales_price#143 * cast(ss_quantity#144 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#143 * cast(ss_quantity#144 as decimal(10,0))), 0.00))#21] -Results [3]: [i_category#133, i_class#134, sum(coalesce((ss_sales_price#143 * cast(ss_quantity#144 as decimal(10,0))), 0.00))#21 AS sumsales#145] - -(50) HashAggregate [codegen id : 19] -Input [3]: [i_category#133, i_class#134, sumsales#145] -Keys [2]: [i_category#133, i_class#134] -Functions [1]: [partial_sum(sumsales#145)] -Aggregate Attributes [2]: [sum#146, isEmpty#147] -Results [4]: [i_category#133, i_class#134, sum#148, isEmpty#149] - -(51) Exchange -Input [4]: [i_category#133, i_class#134, sum#148, isEmpty#149] -Arguments: hashpartitioning(i_category#133, i_class#134, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(52) HashAggregate [codegen id : 20] -Input [4]: [i_category#133, i_class#134, sum#148, isEmpty#149] -Keys [2]: [i_category#133, i_class#134] -Functions [1]: [sum(sumsales#145)] -Aggregate Attributes [1]: [sum(sumsales#145)#150] -Results [9]: [i_category#133, i_class#134, null AS i_brand#151, null AS i_product_name#152, null AS d_year#153, null AS d_qoy#154, null AS d_moy#155, null AS s_store_id#156, sum(sumsales#145)#150 AS sumsales#157] - -(53) ReusedExchange [Reuses operator id: 21] -Output [10]: [i_category#158, i_class#159, i_brand#160, i_product_name#161, d_year#162, d_qoy#163, d_moy#164, s_store_id#165, sum#166, isEmpty#167] - -(54) HashAggregate [codegen id : 22] -Input [10]: [i_category#158, i_class#159, i_brand#160, i_product_name#161, d_year#162, d_qoy#163, d_moy#164, s_store_id#165, sum#166, isEmpty#167] -Keys [8]: [i_category#158, i_class#159, i_brand#160, i_product_name#161, d_year#162, d_qoy#163, d_moy#164, s_store_id#165] -Functions [1]: [sum(coalesce((ss_sales_price#168 * cast(ss_quantity#169 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#168 * cast(ss_quantity#169 as decimal(10,0))), 0.00))#21] -Results [2]: [i_category#158, sum(coalesce((ss_sales_price#168 * cast(ss_quantity#169 as decimal(10,0))), 0.00))#21 AS sumsales#170] - -(55) HashAggregate [codegen id : 22] -Input [2]: [i_category#158, sumsales#170] -Keys [1]: [i_category#158] -Functions [1]: [partial_sum(sumsales#170)] -Aggregate Attributes [2]: [sum#171, isEmpty#172] -Results [3]: [i_category#158, sum#173, isEmpty#174] - -(56) Exchange -Input [3]: [i_category#158, sum#173, isEmpty#174] -Arguments: hashpartitioning(i_category#158, 5), ENSURE_REQUIREMENTS, [plan_id=8] - -(57) HashAggregate [codegen id : 23] -Input [3]: [i_category#158, sum#173, isEmpty#174] -Keys [1]: [i_category#158] -Functions [1]: [sum(sumsales#170)] -Aggregate Attributes [1]: [sum(sumsales#170)#175] -Results [9]: [i_category#158, null AS i_class#176, null AS i_brand#177, null AS i_product_name#178, null AS d_year#179, null AS d_qoy#180, null AS d_moy#181, null AS s_store_id#182, sum(sumsales#170)#175 AS sumsales#183] - -(58) ReusedExchange [Reuses operator id: 21] -Output [10]: [i_category#184, i_class#185, i_brand#186, i_product_name#187, d_year#188, d_qoy#189, d_moy#190, s_store_id#191, sum#192, isEmpty#193] - -(59) HashAggregate [codegen id : 25] -Input [10]: [i_category#184, i_class#185, i_brand#186, i_product_name#187, d_year#188, d_qoy#189, d_moy#190, s_store_id#191, sum#192, isEmpty#193] -Keys [8]: [i_category#184, i_class#185, i_brand#186, i_product_name#187, d_year#188, d_qoy#189, d_moy#190, s_store_id#191] -Functions [1]: [sum(coalesce((ss_sales_price#194 * cast(ss_quantity#195 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#194 * cast(ss_quantity#195 as decimal(10,0))), 0.00))#21] -Results [1]: [sum(coalesce((ss_sales_price#194 * cast(ss_quantity#195 as decimal(10,0))), 0.00))#21 AS sumsales#196] - -(60) HashAggregate [codegen id : 25] -Input [1]: [sumsales#196] + +(22) ReusedExchange [Reuses operator id: 20] +Output [10]: [i_category#21, i_class#22, i_brand#23, i_product_name#24, d_year#25, d_qoy#26, d_moy#27, s_store_id#28, sum#29, isEmpty#30] + +(23) CometHashAggregate +Input [10]: [i_category#21, i_class#22, i_brand#23, i_product_name#24, d_year#25, d_qoy#26, d_moy#27, s_store_id#28, sum#29, isEmpty#30] +Keys [8]: [i_category#21, i_class#22, i_brand#23, i_product_name#24, d_year#25, d_qoy#26, d_moy#27, s_store_id#28] +Functions [1]: [sum(coalesce((ss_sales_price#31 * cast(ss_quantity#32 as decimal(10,0))), 0.00))] + +(24) CometHashAggregate +Input [8]: [i_category#21, i_class#22, i_brand#23, i_product_name#24, d_year#25, d_qoy#26, d_moy#27, sumsales#33] +Keys [7]: [i_category#21, i_class#22, i_brand#23, i_product_name#24, d_year#25, d_qoy#26, d_moy#27] +Functions [1]: [partial_sum(sumsales#33)] + +(25) CometColumnarExchange +Input [9]: [i_category#21, i_class#22, i_brand#23, i_product_name#24, d_year#25, d_qoy#26, d_moy#27, sum#34, isEmpty#35] +Arguments: hashpartitioning(i_category#21, i_class#22, i_brand#23, i_product_name#24, d_year#25, d_qoy#26, d_moy#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(26) CometHashAggregate +Input [9]: [i_category#21, i_class#22, i_brand#23, i_product_name#24, d_year#25, d_qoy#26, d_moy#27, sum#34, isEmpty#35] +Keys [7]: [i_category#21, i_class#22, i_brand#23, i_product_name#24, d_year#25, d_qoy#26, d_moy#27] +Functions [1]: [sum(sumsales#33)] + +(27) ReusedExchange [Reuses operator id: 20] +Output [10]: [i_category#36, i_class#37, i_brand#38, i_product_name#39, d_year#40, d_qoy#41, d_moy#42, s_store_id#43, sum#44, isEmpty#45] + +(28) CometHashAggregate +Input [10]: [i_category#36, i_class#37, i_brand#38, i_product_name#39, d_year#40, d_qoy#41, d_moy#42, s_store_id#43, sum#44, isEmpty#45] +Keys [8]: [i_category#36, i_class#37, i_brand#38, i_product_name#39, d_year#40, d_qoy#41, d_moy#42, s_store_id#43] +Functions [1]: [sum(coalesce((ss_sales_price#46 * cast(ss_quantity#47 as decimal(10,0))), 0.00))] + +(29) CometHashAggregate +Input [7]: [i_category#36, i_class#37, i_brand#38, i_product_name#39, d_year#40, d_qoy#41, sumsales#48] +Keys [6]: [i_category#36, i_class#37, i_brand#38, i_product_name#39, d_year#40, d_qoy#41] +Functions [1]: [partial_sum(sumsales#48)] + +(30) CometColumnarExchange +Input [8]: [i_category#36, i_class#37, i_brand#38, i_product_name#39, d_year#40, d_qoy#41, sum#49, isEmpty#50] +Arguments: hashpartitioning(i_category#36, i_class#37, i_brand#38, i_product_name#39, d_year#40, d_qoy#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(31) CometHashAggregate +Input [8]: [i_category#36, i_class#37, i_brand#38, i_product_name#39, d_year#40, d_qoy#41, sum#49, isEmpty#50] +Keys [6]: [i_category#36, i_class#37, i_brand#38, i_product_name#39, d_year#40, d_qoy#41] +Functions [1]: [sum(sumsales#48)] + +(32) ReusedExchange [Reuses operator id: 20] +Output [10]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, d_moy#57, s_store_id#58, sum#59, isEmpty#60] + +(33) CometHashAggregate +Input [10]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, d_moy#57, s_store_id#58, sum#59, isEmpty#60] +Keys [8]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, d_moy#57, s_store_id#58] +Functions [1]: [sum(coalesce((ss_sales_price#61 * cast(ss_quantity#62 as decimal(10,0))), 0.00))] + +(34) CometHashAggregate +Input [6]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, sumsales#63] +Keys [5]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55] +Functions [1]: [partial_sum(sumsales#63)] + +(35) CometColumnarExchange +Input [7]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, sum#64, isEmpty#65] +Arguments: hashpartitioning(i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(36) CometHashAggregate +Input [7]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, sum#64, isEmpty#65] +Keys [5]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55] +Functions [1]: [sum(sumsales#63)] + +(37) ReusedExchange [Reuses operator id: 20] +Output [10]: [i_category#66, i_class#67, i_brand#68, i_product_name#69, d_year#70, d_qoy#71, d_moy#72, s_store_id#73, sum#74, isEmpty#75] + +(38) CometHashAggregate +Input [10]: [i_category#66, i_class#67, i_brand#68, i_product_name#69, d_year#70, d_qoy#71, d_moy#72, s_store_id#73, sum#74, isEmpty#75] +Keys [8]: [i_category#66, i_class#67, i_brand#68, i_product_name#69, d_year#70, d_qoy#71, d_moy#72, s_store_id#73] +Functions [1]: [sum(coalesce((ss_sales_price#76 * cast(ss_quantity#77 as decimal(10,0))), 0.00))] + +(39) CometHashAggregate +Input [5]: [i_category#66, i_class#67, i_brand#68, i_product_name#69, sumsales#78] +Keys [4]: [i_category#66, i_class#67, i_brand#68, i_product_name#69] +Functions [1]: [partial_sum(sumsales#78)] + +(40) CometColumnarExchange +Input [6]: [i_category#66, i_class#67, i_brand#68, i_product_name#69, sum#79, isEmpty#80] +Arguments: hashpartitioning(i_category#66, i_class#67, i_brand#68, i_product_name#69, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(41) CometHashAggregate +Input [6]: [i_category#66, i_class#67, i_brand#68, i_product_name#69, sum#79, isEmpty#80] +Keys [4]: [i_category#66, i_class#67, i_brand#68, i_product_name#69] +Functions [1]: [sum(sumsales#78)] + +(42) ReusedExchange [Reuses operator id: 20] +Output [10]: [i_category#81, i_class#82, i_brand#83, i_product_name#84, d_year#85, d_qoy#86, d_moy#87, s_store_id#88, sum#89, isEmpty#90] + +(43) CometHashAggregate +Input [10]: [i_category#81, i_class#82, i_brand#83, i_product_name#84, d_year#85, d_qoy#86, d_moy#87, s_store_id#88, sum#89, isEmpty#90] +Keys [8]: [i_category#81, i_class#82, i_brand#83, i_product_name#84, d_year#85, d_qoy#86, d_moy#87, s_store_id#88] +Functions [1]: [sum(coalesce((ss_sales_price#91 * cast(ss_quantity#92 as decimal(10,0))), 0.00))] + +(44) CometHashAggregate +Input [4]: [i_category#81, i_class#82, i_brand#83, sumsales#93] +Keys [3]: [i_category#81, i_class#82, i_brand#83] +Functions [1]: [partial_sum(sumsales#93)] + +(45) CometColumnarExchange +Input [5]: [i_category#81, i_class#82, i_brand#83, sum#94, isEmpty#95] +Arguments: hashpartitioning(i_category#81, i_class#82, i_brand#83, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(46) CometHashAggregate +Input [5]: [i_category#81, i_class#82, i_brand#83, sum#94, isEmpty#95] +Keys [3]: [i_category#81, i_class#82, i_brand#83] +Functions [1]: [sum(sumsales#93)] + +(47) ReusedExchange [Reuses operator id: 20] +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] + +(48) CometHashAggregate +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))] + +(49) CometHashAggregate +Input [3]: [i_category#96, i_class#97, sumsales#108] +Keys [2]: [i_category#96, i_class#97] +Functions [1]: [partial_sum(sumsales#108)] + +(50) CometColumnarExchange +Input [4]: [i_category#96, i_class#97, sum#109, isEmpty#110] +Arguments: hashpartitioning(i_category#96, i_class#97, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(51) CometHashAggregate +Input [4]: [i_category#96, i_class#97, sum#109, isEmpty#110] +Keys [2]: [i_category#96, i_class#97] +Functions [1]: [sum(sumsales#108)] + +(52) ReusedExchange [Reuses operator id: 20] +Output [10]: [i_category#111, i_class#112, i_brand#113, i_product_name#114, d_year#115, d_qoy#116, d_moy#117, s_store_id#118, sum#119, isEmpty#120] + +(53) CometHashAggregate +Input [10]: [i_category#111, i_class#112, i_brand#113, i_product_name#114, d_year#115, d_qoy#116, d_moy#117, s_store_id#118, sum#119, isEmpty#120] +Keys [8]: [i_category#111, i_class#112, i_brand#113, i_product_name#114, d_year#115, d_qoy#116, d_moy#117, s_store_id#118] +Functions [1]: [sum(coalesce((ss_sales_price#121 * cast(ss_quantity#122 as decimal(10,0))), 0.00))] + +(54) CometHashAggregate +Input [2]: [i_category#111, sumsales#123] +Keys [1]: [i_category#111] +Functions [1]: [partial_sum(sumsales#123)] + +(55) CometColumnarExchange +Input [3]: [i_category#111, sum#124, isEmpty#125] +Arguments: hashpartitioning(i_category#111, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(56) CometHashAggregate +Input [3]: [i_category#111, sum#124, isEmpty#125] +Keys [1]: [i_category#111] +Functions [1]: [sum(sumsales#123)] + +(57) ReusedExchange [Reuses operator id: 20] +Output [10]: [i_category#126, i_class#127, i_brand#128, i_product_name#129, d_year#130, d_qoy#131, d_moy#132, s_store_id#133, sum#134, isEmpty#135] + +(58) CometHashAggregate +Input [10]: [i_category#126, i_class#127, i_brand#128, i_product_name#129, d_year#130, d_qoy#131, d_moy#132, s_store_id#133, sum#134, isEmpty#135] +Keys [8]: [i_category#126, i_class#127, i_brand#128, i_product_name#129, d_year#130, d_qoy#131, d_moy#132, s_store_id#133] +Functions [1]: [sum(coalesce((ss_sales_price#136 * cast(ss_quantity#137 as decimal(10,0))), 0.00))] + +(59) CometHashAggregate +Input [1]: [sumsales#138] Keys: [] -Functions [1]: [partial_sum(sumsales#196)] -Aggregate Attributes [2]: [sum#197, isEmpty#198] -Results [2]: [sum#199, isEmpty#200] +Functions [1]: [partial_sum(sumsales#138)] -(61) Exchange -Input [2]: [sum#199, isEmpty#200] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] +(60) CometColumnarExchange +Input [2]: [sum#139, isEmpty#140] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(62) HashAggregate [codegen id : 26] -Input [2]: [sum#199, isEmpty#200] +(61) CometHashAggregate +Input [2]: [sum#139, isEmpty#140] Keys: [] -Functions [1]: [sum(sumsales#196)] -Aggregate Attributes [1]: [sum(sumsales#196)#201] -Results [9]: [null AS i_category#202, null AS i_class#203, null AS i_brand#204, null AS i_product_name#205, null AS d_year#206, null AS d_qoy#207, null AS d_moy#208, null AS s_store_id#209, sum(sumsales#196)#201 AS sumsales#210] +Functions [1]: [sum(sumsales#138)] + +(62) CometUnion +Child 0 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#141] +Child 1 Input [9]: [i_category#21, i_class#22, i_brand#23, i_product_name#24, d_year#25, d_qoy#26, d_moy#27, s_store_id#142, sumsales#143] +Child 2 Input [9]: [i_category#36, i_class#37, i_brand#38, i_product_name#39, d_year#40, d_qoy#41, d_moy#144, s_store_id#145, sumsales#146] +Child 3 Input [9]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#147, d_moy#148, s_store_id#149, sumsales#150] +Child 4 Input [9]: [i_category#66, i_class#67, i_brand#68, i_product_name#69, d_year#151, d_qoy#152, d_moy#153, s_store_id#154, sumsales#155] +Child 5 Input [9]: [i_category#81, i_class#82, i_brand#83, i_product_name#156, d_year#157, d_qoy#158, d_moy#159, s_store_id#160, sumsales#161] +Child 6 Input [9]: [i_category#96, i_class#97, i_brand#162, i_product_name#163, d_year#164, d_qoy#165, d_moy#166, s_store_id#167, sumsales#168] +Child 7 Input [9]: [i_category#111, i_class#169, i_brand#170, i_product_name#171, d_year#172, d_qoy#173, d_moy#174, s_store_id#175, sumsales#176] +Child 8 Input [9]: [i_category#177, i_class#178, i_brand#179, i_product_name#180, d_year#181, d_qoy#182, d_moy#183, s_store_id#184, sumsales#185] + +(63) CometSort +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#141] +Arguments: [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#141], [i_category#17 ASC NULLS FIRST, sumsales#141 DESC NULLS LAST] + +(64) ColumnarToRow [codegen id : 1] +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#141] -(63) Union +(65) WindowGroupLimit +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#141] +Arguments: [i_category#17], [sumsales#141 DESC NULLS LAST], rank(sumsales#141), 100, Partial -(64) 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 +(66) RowToColumnar +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#141] -(65) WindowGroupLimit -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], [sumsales#22 DESC NULLS LAST], rank(sumsales#22), 100, Partial +(67) CometColumnarExchange +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#141] +Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(66) 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] +(68) CometSort +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#141] +Arguments: [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#141], [i_category#17 ASC NULLS FIRST, sumsales#141 DESC NULLS LAST] -(67) Sort [codegen id : 28] -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 +(69) ColumnarToRow [codegen id : 2] +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#141] -(68) WindowGroupLimit -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], [sumsales#22 DESC NULLS LAST], rank(sumsales#22), 100, Final +(70) WindowGroupLimit +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#141] +Arguments: [i_category#17], [sumsales#141 DESC NULLS LAST], rank(sumsales#141), 100, Final -(69) 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#211], [i_category#17], [sumsales#22 DESC NULLS LAST] +(71) 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#141] +Arguments: [rank(sumsales#141) windowspecdefinition(i_category#17, sumsales#141 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#186], [i_category#17], [sumsales#141 DESC NULLS LAST] -(70) Filter [codegen id : 29] -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#211] -Condition : (rk#211 <= 100) +(72) Filter [codegen id : 3] +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#141, rk#186] +Condition : (rk#186 <= 100) -(71) 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#211] -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#211 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#211] +(73) 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#141, rk#186] +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#141 ASC NULLS FIRST, rk#186 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#141, rk#186] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (76) -+- * ColumnarToRow (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan parquet spark_catalog.default.date_dim (72) +BroadcastExchange (78) ++- * ColumnarToRow (77) + +- CometProject (76) + +- CometFilter (75) + +- CometScan parquet spark_catalog.default.date_dim (74) -(72) Scan parquet spark_catalog.default.date_dim +(74) 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 -(73) CometFilter +(75) 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)) -(74) CometProject +(76) 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] -(75) ColumnarToRow [codegen id : 1] +(77) ColumnarToRow [codegen id : 1] Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(76) BroadcastExchange +(78) 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-spark3_5/q67a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/simplified.txt index 5838786d4..2298c66db 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/simplified.txt @@ -1,121 +1,87 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (29) + WholeStageCodegen (3) Filter [rk] InputAdapter Window [sumsales,i_category] WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (28) - Sort [i_category,sumsales] + WholeStageCodegen (2) + ColumnarToRow InputAdapter - Exchange [i_category] #1 - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (27) - Sort [i_category,sumsales] - InputAdapter - Union - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty,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,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] - CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] - CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometFilter [s_store_sk,s_store_id] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - 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] #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 (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] #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 (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] #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 (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] #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 (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] #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 (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] #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 (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] #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 (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 #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 - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometColumnarExchange [i_category] #1 + RowToColumnar + WindowGroupLimit [i_category,sumsales] + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometUnion [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty,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,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] + CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty,sum(sumsales)] + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #7 + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty,sumsales] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales,s_store_id,sum,isEmpty,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty,sum(sumsales)] + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #8 + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty,sumsales] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales,d_moy,s_store_id,sum,isEmpty,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty,sum(sumsales)] + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year] #9 + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty,sumsales] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sumsales,d_qoy,d_moy,s_store_id,sum,isEmpty,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty,sum(sumsales)] + CometColumnarExchange [i_category,i_class,i_brand,i_product_name] #10 + CometHashAggregate [i_category,i_class,i_brand,i_product_name,sum,isEmpty,sumsales] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,sumsales,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))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty,sum(sumsales)] + CometColumnarExchange [i_category,i_class,i_brand] #11 + CometHashAggregate [i_category,i_class,i_brand,sum,isEmpty,sumsales] + CometHashAggregate [i_category,i_class,i_brand,sumsales,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))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty,sum(sumsales)] + CometColumnarExchange [i_category,i_class] #12 + CometHashAggregate [i_category,i_class,sum,isEmpty,sumsales] + CometHashAggregate [i_category,i_class,sumsales,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))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty,sum(sumsales)] + CometColumnarExchange [i_category] #13 + CometHashAggregate [i_category,sum,isEmpty,sumsales] + CometHashAggregate [i_category,sumsales,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))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty,sum(sumsales)] + CometColumnarExchange #14 + CometHashAggregate [sum,isEmpty,sumsales] + CometHashAggregate [sumsales,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))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/explain.txt index bb823ddbe..632d097a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/explain.txt @@ -1,62 +1,72 @@ == Physical Plan == -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) - : +- * 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 (44) - : +- Exchange (43) - : +- * HashAggregate (42) - : +- * HashAggregate (41) - : +- ReusedExchange (40) - +- * HashAggregate (49) - +- Exchange (48) - +- * HashAggregate (47) - +- * HashAggregate (46) - +- ReusedExchange (45) +TakeOrderedAndProject (68) ++- * Project (67) + +- Window (66) + +- * ColumnarToRow (65) + +- CometSort (64) + +- CometColumnarExchange (63) + +- CometHashAggregate (62) + +- CometColumnarExchange (61) + +- RowToColumnar (60) + +- * HashAggregate (59) + +- Union (58) + :- * HashAggregate (41) + : +- * ColumnarToRow (40) + : +- CometColumnarExchange (39) + : +- RowToColumnar (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) + : +- * ColumnarToRow (27) + : +- CometSort (26) + : +- CometHashAggregate (25) + : +- CometColumnarExchange (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 (49) + : +- * ColumnarToRow (48) + : +- CometColumnarExchange (47) + : +- RowToColumnar (46) + : +- * HashAggregate (45) + : +- * HashAggregate (44) + : +- * ColumnarToRow (43) + : +- ReusedExchange (42) + +- * HashAggregate (57) + +- * ColumnarToRow (56) + +- CometColumnarExchange (55) + +- RowToColumnar (54) + +- * HashAggregate (53) + +- * HashAggregate (52) + +- * ColumnarToRow (51) + +- ReusedExchange (50) (1) Scan parquet spark_catalog.default.store_sales @@ -99,7 +109,7 @@ Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight 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] +(9) ColumnarToRow [codegen id : 4] Input [2]: [ss_store_sk#1, ss_net_profit#2] (10) Scan parquet spark_catalog.default.store @@ -113,7 +123,7 @@ ReadSchema: struct Input [3]: [s_store_sk#7, s_county#8, s_state#9] Condition : isnotnull(s_store_sk#7) -(12) ColumnarToRow [codegen id : 4] +(12) ColumnarToRow [codegen id : 3] Input [3]: [s_store_sk#7, s_county#8, s_state#9] (13) Scan parquet spark_catalog.default.store_sales @@ -169,45 +179,43 @@ 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] +(24) CometColumnarExchange Input [2]: [s_state#15, sum#17] +Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(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] +(25) CometHashAggregate 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 +(26) CometSort +Input [3]: [s_state#15, _w0#18, s_state#15] +Arguments: [s_state#15, _w0#18, s_state#15], [s_state#15 ASC NULLS FIRST, _w0#18 DESC NULLS LAST] + +(27) ColumnarToRow [codegen id : 1] +Input [3]: [s_state#15, _w0#18, s_state#15] (28) WindowGroupLimit -Input [3]: [s_state#15, _w0#19, s_state#15] -Arguments: [s_state#15], [_w0#19 DESC NULLS LAST], rank(_w0#19), 5, Final +Input [3]: [s_state#15, _w0#18, s_state#15] +Arguments: [s_state#15], [_w0#18 DESC NULLS LAST], rank(_w0#18), 5, Final (29) 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] +Input [3]: [s_state#15, _w0#18, s_state#15] +Arguments: [rank(_w0#18) windowspecdefinition(s_state#15, _w0#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#19], [s_state#15], [_w0#18 DESC NULLS LAST] -(30) Filter [codegen id : 3] -Input [4]: [s_state#15, _w0#19, s_state#15, ranking#20] -Condition : (ranking#20 <= 5) +(30) Filter [codegen id : 2] +Input [4]: [s_state#15, _w0#18, s_state#15, ranking#19] +Condition : (ranking#19 <= 5) -(31) Project [codegen id : 3] +(31) Project [codegen id : 2] Output [1]: [s_state#15] -Input [4]: [s_state#15, _w0#19, s_state#15, ranking#20] +Input [4]: [s_state#15, _w0#18, s_state#15, ranking#19] (32) BroadcastExchange Input [1]: [s_state#15] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] -(33) BroadcastHashJoin [codegen id : 4] +(33) BroadcastHashJoin [codegen id : 3] Left keys [1]: [s_state#9] Right keys [1]: [s_state#15] Join type: LeftSemi @@ -217,159 +225,187 @@ Join condition: None 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] -(35) BroadcastHashJoin [codegen id : 5] +(35) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_store_sk#1] Right keys [1]: [s_store_sk#7] Join type: Inner Join condition: None -(36) Project [codegen id : 5] +(36) Project [codegen id : 4] 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] -(37) HashAggregate [codegen id : 5] +(37) HashAggregate [codegen id : 4] 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#9, s_county#8, sum#22] +Aggregate Attributes [1]: [sum#20] +Results [3]: [s_state#9, s_county#8, sum#21] + +(38) RowToColumnar +Input [3]: [s_state#9, s_county#8, sum#21] + +(39) CometColumnarExchange +Input [3]: [s_state#9, s_county#8, sum#21] +Arguments: hashpartitioning(s_state#9, s_county#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(38) 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] +(40) ColumnarToRow [codegen id : 5] +Input [3]: [s_state#9, s_county#8, sum#21] -(39) HashAggregate [codegen id : 6] -Input [3]: [s_state#9, s_county#8, sum#22] +(41) HashAggregate [codegen id : 5] +Input [3]: [s_state#9, s_county#8, sum#21] 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#9, s_county#8, 0 AS g_state#25, 0 AS g_county#26, 0 AS lochierarchy#27] - -(40) ReusedExchange [Reuses operator id: 38] -Output [3]: [s_state#28, s_county#29, sum#30] - -(41) HashAggregate [codegen id : 12] -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] - -(42) HashAggregate [codegen id : 12] -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] - -(43) Exchange -Input [3]: [s_state#28, sum#35, isEmpty#36] -Arguments: hashpartitioning(s_state#28, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(44) HashAggregate [codegen id : 13] -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] - -(45) ReusedExchange [Reuses operator id: 38] -Output [3]: [s_state#43, s_county#44, sum#45] - -(46) HashAggregate [codegen id : 19] -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] - -(47) HashAggregate [codegen id : 19] -Input [1]: [total_sum#47] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#22] +Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#22,17,2) as decimal(27,2)) AS total_sum#23, s_state#9, s_county#8, 0 AS g_state#24, 0 AS g_county#25, 0 AS lochierarchy#26] + +(42) ReusedExchange [Reuses operator id: 39] +Output [3]: [s_state#27, s_county#28, sum#29] + +(43) ColumnarToRow [codegen id : 10] +Input [3]: [s_state#27, s_county#28, sum#29] + +(44) HashAggregate [codegen id : 10] +Input [3]: [s_state#27, s_county#28, sum#29] +Keys [2]: [s_state#27, s_county#28] +Functions [1]: [sum(UnscaledValue(ss_net_profit#30))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#30))#22] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#30))#22,17,2) AS total_sum#31, s_state#27] + +(45) HashAggregate [codegen id : 10] +Input [2]: [total_sum#31, s_state#27] +Keys [1]: [s_state#27] +Functions [1]: [partial_sum(total_sum#31)] +Aggregate Attributes [2]: [sum#32, isEmpty#33] +Results [3]: [s_state#27, sum#34, isEmpty#35] + +(46) RowToColumnar +Input [3]: [s_state#27, sum#34, isEmpty#35] + +(47) CometColumnarExchange +Input [3]: [s_state#27, sum#34, isEmpty#35] +Arguments: hashpartitioning(s_state#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(48) ColumnarToRow [codegen id : 11] +Input [3]: [s_state#27, sum#34, isEmpty#35] + +(49) HashAggregate [codegen id : 11] +Input [3]: [s_state#27, sum#34, isEmpty#35] +Keys [1]: [s_state#27] +Functions [1]: [sum(total_sum#31)] +Aggregate Attributes [1]: [sum(total_sum#31)#36] +Results [6]: [sum(total_sum#31)#36 AS total_sum#37, s_state#27, null AS s_county#38, 0 AS g_state#39, 1 AS g_county#40, 1 AS lochierarchy#41] + +(50) ReusedExchange [Reuses operator id: 39] +Output [3]: [s_state#42, s_county#43, sum#44] + +(51) ColumnarToRow [codegen id : 16] +Input [3]: [s_state#42, s_county#43, sum#44] + +(52) HashAggregate [codegen id : 16] +Input [3]: [s_state#42, s_county#43, sum#44] +Keys [2]: [s_state#42, s_county#43] +Functions [1]: [sum(UnscaledValue(ss_net_profit#45))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#45))#22] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#45))#22,17,2) AS total_sum#46] + +(53) HashAggregate [codegen id : 16] +Input [1]: [total_sum#46] 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#46)] +Aggregate Attributes [2]: [sum#47, isEmpty#48] +Results [2]: [sum#49, isEmpty#50] -(48) Exchange -Input [2]: [sum#50, isEmpty#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +(54) RowToColumnar +Input [2]: [sum#49, isEmpty#50] -(49) HashAggregate [codegen id : 20] -Input [2]: [sum#50, isEmpty#51] +(55) CometColumnarExchange +Input [2]: [sum#49, isEmpty#50] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(56) ColumnarToRow [codegen id : 17] +Input [2]: [sum#49, isEmpty#50] + +(57) HashAggregate [codegen id : 17] +Input [2]: [sum#49, isEmpty#50] 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#46)] +Aggregate Attributes [1]: [sum(total_sum#46)#51] +Results [6]: [sum(total_sum#46)#51 AS total_sum#52, null AS s_state#53, null AS s_county#54, 1 AS g_state#55, 1 AS g_county#56, 2 AS lochierarchy#57] -(50) Union +(58) Union -(51) 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] +(59) HashAggregate [codegen id : 18] +Input [6]: [total_sum#23, s_state#9, s_county#8, g_state#24, g_county#25, lochierarchy#26] +Keys [6]: [total_sum#23, s_state#9, s_county#8, g_state#24, g_county#25, lochierarchy#26] Functions: [] Aggregate Attributes: [] -Results [6]: [total_sum#24, s_state#9, s_county#8, g_state#25, g_county#26, lochierarchy#27] +Results [6]: [total_sum#23, s_state#9, s_county#8, g_state#24, g_county#25, lochierarchy#26] + +(60) RowToColumnar +Input [6]: [total_sum#23, s_state#9, s_county#8, g_state#24, g_county#25, lochierarchy#26] -(52) 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] +(61) CometColumnarExchange +Input [6]: [total_sum#23, s_state#9, s_county#8, g_state#24, g_county#25, lochierarchy#26] +Arguments: hashpartitioning(total_sum#23, s_state#9, s_county#8, g_state#24, g_county#25, lochierarchy#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(53) 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] +(62) CometHashAggregate +Input [6]: [total_sum#23, s_state#9, s_county#8, g_state#24, g_county#25, lochierarchy#26] +Keys [6]: [total_sum#23, s_state#9, s_county#8, g_state#24, g_county#25, lochierarchy#26] Functions: [] -Aggregate Attributes: [] -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#59] -(54) Exchange -Input [5]: [total_sum#24, s_state#9, s_county#8, lochierarchy#27, _w0#59] -Arguments: hashpartitioning(lochierarchy#27, _w0#59, 5), ENSURE_REQUIREMENTS, [plan_id=8] +(63) CometColumnarExchange +Input [5]: [total_sum#23, s_state#9, s_county#8, lochierarchy#26, _w0#58] +Arguments: hashpartitioning(lochierarchy#26, _w0#58, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(64) CometSort +Input [5]: [total_sum#23, s_state#9, s_county#8, lochierarchy#26, _w0#58] +Arguments: [total_sum#23, s_state#9, s_county#8, lochierarchy#26, _w0#58], [lochierarchy#26 ASC NULLS FIRST, _w0#58 ASC NULLS FIRST, total_sum#23 DESC NULLS LAST] -(55) Sort [codegen id : 23] -Input [5]: [total_sum#24, s_state#9, s_county#8, lochierarchy#27, _w0#59] -Arguments: [lochierarchy#27 ASC NULLS FIRST, _w0#59 ASC NULLS FIRST, total_sum#24 DESC NULLS LAST], false, 0 +(65) ColumnarToRow [codegen id : 19] +Input [5]: [total_sum#23, s_state#9, s_county#8, lochierarchy#26, _w0#58] -(56) Window -Input [5]: [total_sum#24, s_state#9, s_county#8, 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] +(66) Window +Input [5]: [total_sum#23, s_state#9, s_county#8, lochierarchy#26, _w0#58] +Arguments: [rank(total_sum#23) windowspecdefinition(lochierarchy#26, _w0#58, total_sum#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#59], [lochierarchy#26, _w0#58], [total_sum#23 DESC NULLS LAST] -(57) Project [codegen id : 24] -Output [5]: [total_sum#24, s_state#9, s_county#8, lochierarchy#27, rank_within_parent#60] -Input [6]: [total_sum#24, s_state#9, s_county#8, lochierarchy#27, _w0#59, rank_within_parent#60] +(67) Project [codegen id : 20] +Output [5]: [total_sum#23, s_state#9, s_county#8, lochierarchy#26, rank_within_parent#59] +Input [6]: [total_sum#23, s_state#9, s_county#8, lochierarchy#26, _w0#58, rank_within_parent#59] -(58) TakeOrderedAndProject -Input [5]: [total_sum#24, s_state#9, s_county#8, lochierarchy#27, rank_within_parent#60] -Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN s_state#9 END ASC NULLS FIRST, rank_within_parent#60 ASC NULLS FIRST], [total_sum#24, s_state#9, s_county#8, lochierarchy#27, rank_within_parent#60] +(68) TakeOrderedAndProject +Input [5]: [total_sum#23, s_state#9, s_county#8, lochierarchy#26, rank_within_parent#59] +Arguments: 100, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0) THEN s_state#9 END ASC NULLS FIRST, rank_within_parent#59 ASC NULLS FIRST], [total_sum#23, s_state#9, s_county#8, lochierarchy#26, rank_within_parent#59] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (63) -+- * ColumnarToRow (62) - +- CometProject (61) - +- CometFilter (60) - +- CometScan parquet spark_catalog.default.date_dim (59) +BroadcastExchange (73) ++- * ColumnarToRow (72) + +- CometProject (71) + +- CometFilter (70) + +- CometScan parquet spark_catalog.default.date_dim (69) -(59) Scan parquet spark_catalog.default.date_dim +(69) 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 -(60) CometFilter +(70) 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)) -(61) CometProject +(71) CometProject Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(62) ColumnarToRow [codegen id : 1] +(72) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(63) BroadcastExchange +(73) BroadcastExchange Input [1]: [d_date_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/simplified.txt index e76a524db..5c57b1764 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/simplified.txt @@ -1,70 +1,70 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (24) + WholeStageCodegen (20) Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] InputAdapter Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (23) - Sort [lochierarchy,_w0,total_sum] + WholeStageCodegen (19) + ColumnarToRow InputAdapter - Exchange [lochierarchy,_w0] #1 - 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 (21) + CometSort [total_sum,s_state,s_county,lochierarchy,_w0] + CometColumnarExchange [lochierarchy,_w0] #1 + CometHashAggregate [total_sum,s_state,s_county,lochierarchy,_w0,g_state,g_county] + CometColumnarExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 + RowToColumnar + WholeStageCodegen (18) HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] InputAdapter Union - WholeStageCodegen (6) + WholeStageCodegen (5) 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 (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] - ColumnarToRow - InputAdapter - CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - BroadcastHashJoin [s_state,s_state] - ColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (2) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - InputAdapter - Exchange [s_state] #8 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometColumnarExchange [s_state,s_county] #3 + RowToColumnar + WholeStageCodegen (4) + 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] + ColumnarToRow + InputAdapter + CometProject [ss_store_sk,ss_net_profit] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + BroadcastHashJoin [s_state,s_state] + ColumnarToRow + InputAdapter + CometFilter [s_store_sk,s_county,s_state] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (2) + Project [s_state] + Filter [ranking] + InputAdapter + Window [_w0,s_state] + WindowGroupLimit [s_state,_w0] + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometSort [s_state,_w0] + CometHashAggregate [s_state,_w0,sum,sum(UnscaledValue(ss_net_profit))] + CometColumnarExchange [s_state] #8 CometHashAggregate [s_state,sum,ss_net_profit] CometProject [ss_net_profit,s_state] CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] @@ -77,21 +77,27 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [d_date_sk] #5 - WholeStageCodegen (13) + WholeStageCodegen (11) HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - InputAdapter - 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 (20) + ColumnarToRow + InputAdapter + CometColumnarExchange [s_state] #10 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] + ColumnarToRow + InputAdapter + ReusedExchange [s_state,s_county,sum] #3 + WholeStageCodegen (17) HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - InputAdapter - 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 - ReusedExchange [s_state,s_county,sum] #3 + ColumnarToRow + InputAdapter + CometColumnarExchange #11 + RowToColumnar + WholeStageCodegen (16) + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] + ColumnarToRow + InputAdapter + ReusedExchange [s_state,s_county,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/explain.txt index 8bff19a72..53e1bb5a6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/explain.txt @@ -1,71 +1,72 @@ == Physical Plan == -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) +* ColumnarToRow (68) ++- CometTakeOrderedAndProject (67) + +- CometHashAggregate (66) + +- CometColumnarExchange (65) + +- CometHashAggregate (64) + +- CometProject (63) + +- CometSortMergeJoin (62) + :- CometSort (56) + : +- CometColumnarExchange (55) + : +- RowToColumnar (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) + +- CometSort (61) + +- CometColumnarExchange (60) + +- CometProject (59) + +- CometFilter (58) + +- CometScan parquet spark_catalog.default.catalog_returns (57) (1) Scan parquet spark_catalog.default.catalog_sales @@ -320,101 +321,99 @@ Join condition: None 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#30] -(54) Exchange +(54) RowToColumnar 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=3] -(55) Sort [codegen id : 4] +(55) CometColumnarExchange 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 +Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(56) Scan parquet spark_catalog.default.catalog_returns +(56) CometSort +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, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST] + +(57) 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 -(57) CometFilter +(58) 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)) -(58) CometProject +(59) 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] -(59) ColumnarToRow [codegen id : 5] +(60) CometColumnarExchange Input [2]: [cr_item_sk#31, cr_order_number#32] +Arguments: hashpartitioning(cr_item_sk#31, cr_order_number#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(60) Exchange +(61) CometSort 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] +Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31 ASC NULLS FIRST, cr_order_number#32 ASC NULLS FIRST] -(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 +(62) CometSortMergeJoin +Left output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +Right output [2]: [cr_item_sk#31, cr_order_number#32] +Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#31, cr_order_number#32], LeftOuter -(62) SortMergeJoin [codegen id : 7] -Left keys [2]: [cs_item_sk#4, cs_order_number#6] -Right keys [2]: [cr_item_sk#31, cr_order_number#32] -Join type: LeftOuter -Join condition: None - -(63) Project [codegen id : 7] -Output [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +(63) CometProject 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] +Arguments: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -(64) HashAggregate [codegen id : 7] +(64) CometHashAggregate 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#34] -Results [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#35] -(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] +(65) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=5] -(66) HashAggregate [codegen id : 8] -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#35] +(66) CometHashAggregate +Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] Functions [1]: [count(1)] -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] -(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] +(67) CometTakeOrderedAndProject +Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#37 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#35,promo#36,total_cnt#37]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37], 100, [total_cnt#37 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#35, promo#36, total_cnt#37] + +(68) ColumnarToRow [codegen id : 4] +Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -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 +(69) 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 -(69) CometFilter +(70) 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)) -(70) CometProject +(71) 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] -(71) ColumnarToRow [codegen id : 1] +(72) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -(72) BroadcastExchange +(73) 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=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/simplified.txt index 6cba2d0e6..1022e1a90 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/simplified.txt @@ -1,94 +1,86 @@ -TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo] - 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 (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 (4) - Sort [cs_item_sk,cs_order_number] - InputAdapter - Exchange [cs_item_sk,cs_order_number] #2 - 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] - 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 [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,d_date_sk,d_week_seq] - 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_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,d_date_sk,d_date,d_week_seq] - 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_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,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_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,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_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_sk,i_item_desc] - 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 [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,w_warehouse_sk,w_warehouse_name] - 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_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,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometFilter [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] - 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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [i_item_sk,i_item_desc] #6 - CometFilter [i_item_sk,i_item_desc] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange [cd_demo_sk] #7 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange [hd_demo_sk] #8 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange [d_date_sk,d_week_seq] #10 - CometFilter [d_date_sk,d_week_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] +WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] + CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count,count(1)] + CometColumnarExchange [i_item_desc,w_warehouse_name,d_week_seq] #1 + CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] + CometProject [w_warehouse_name,i_item_desc,d_week_seq] + CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] + CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + CometColumnarExchange [cs_item_sk,cs_order_number] #2 + RowToColumnar + 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] + 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 [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,d_date_sk,d_week_seq] + 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_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,d_date_sk,d_date,d_week_seq] + 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_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,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_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,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_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_sk,i_item_desc] + 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 [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,w_warehouse_sk,w_warehouse_name] + 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_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,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometFilter [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] + 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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange [i_item_sk,i_item_desc] #6 + CometFilter [i_item_sk,i_item_desc] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + CometBroadcastExchange [cd_demo_sk] #7 + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + CometBroadcastExchange [hd_demo_sk] #8 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] + CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + CometBroadcastExchange [d_date_sk,d_week_seq] #10 + CometFilter [d_date_sk,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter - BroadcastExchange #12 - WholeStageCodegen (2) + BroadcastExchange #11 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk] - InputAdapter - WholeStageCodegen (6) - Sort [cr_item_sk,cr_order_number] - InputAdapter - Exchange [cr_item_sk,cr_order_number] #13 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk] + CometSort [cr_item_sk,cr_order_number] + CometColumnarExchange [cr_item_sk,cr_order_number] #13 + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/explain.txt index 661c552f2..e92f554c5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/explain.txt @@ -1,75 +1,72 @@ == Physical Plan == -TakeOrderedAndProject (71) -+- * Project (70) - +- * BroadcastHashJoin Inner BuildRight (69) - :- * 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) - +- * 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) +* ColumnarToRow (68) ++- CometTakeOrderedAndProject (67) + +- CometProject (66) + +- CometBroadcastHashJoin (65) + :- CometProject (50) + : +- CometBroadcastHashJoin (49) + : :- CometBroadcastHashJoin (33) + : : :- CometFilter (16) + : : : +- CometHashAggregate (15) + : : : +- CometColumnarExchange (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) + : : +- CometBroadcastExchange (32) + : : +- CometHashAggregate (31) + : : +- CometColumnarExchange (30) + : : +- CometHashAggregate (29) + : : +- CometProject (28) + : : +- CometBroadcastHashJoin (27) + : : :- CometProject (23) + : : : +- CometBroadcastHashJoin (22) + : : : :- CometFilter (18) + : : : : +- CometScan parquet spark_catalog.default.customer (17) + : : : +- CometBroadcastExchange (21) + : : : +- CometFilter (20) + : : : +- CometScan parquet spark_catalog.default.store_sales (19) + : : +- CometBroadcastExchange (26) + : : +- CometFilter (25) + : : +- CometScan parquet spark_catalog.default.date_dim (24) + : +- CometBroadcastExchange (48) + : +- CometFilter (47) + : +- CometHashAggregate (46) + : +- CometColumnarExchange (45) + : +- CometHashAggregate (44) + : +- CometProject (43) + : +- CometBroadcastHashJoin (42) + : :- CometProject (40) + : : +- CometBroadcastHashJoin (39) + : : :- CometFilter (35) + : : : +- CometScan parquet spark_catalog.default.customer (34) + : : +- CometBroadcastExchange (38) + : : +- CometFilter (37) + : : +- CometScan parquet spark_catalog.default.web_sales (36) + : +- ReusedExchange (41) + +- CometBroadcastExchange (64) + +- CometHashAggregate (63) + +- CometColumnarExchange (62) + +- CometHashAggregate (61) + +- CometProject (60) + +- CometBroadcastHashJoin (59) + :- CometProject (57) + : +- CometBroadcastHashJoin (56) + : :- CometFilter (52) + : : +- CometScan parquet spark_catalog.default.customer (51) + : +- CometBroadcastExchange (55) + : +- CometFilter (54) + : +- CometScan parquet spark_catalog.default.web_sales (53) + +- ReusedExchange (58) (1) Scan parquet spark_catalog.default.customer @@ -137,337 +134,317 @@ Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_yea 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))] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(16) HashAggregate [codegen id : 8] +(15) CometHashAggregate 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))#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] -(17) Filter [codegen id : 8] -Input [2]: [customer_id#13, year_total#14] -Condition : (isnotnull(year_total#14) AND (year_total#14 > 0.00)) +(16) CometFilter +Input [2]: [customer_id#12, year_total#13] +Condition : (isnotnull(year_total#13) AND (year_total#13 > 0.00)) -(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] +(17) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#14, c_customer_id#15, c_first_name#16, c_last_name#17] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(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)) +(18) CometFilter +Input [4]: [c_customer_sk#14, c_customer_id#15, c_first_name#16, c_last_name#17] +Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_customer_id#15)) -(20) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] +(19) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_customer_sk#18, ss_net_paid#19, ss_sold_date_sk#20] 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#20), dynamicpruningexpression(ss_sold_date_sk#20 IN dynamicpruning#21)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(21) CometFilter -Input [3]: [ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] -Condition : isnotnull(ss_customer_sk#19) +(20) CometFilter +Input [3]: [ss_customer_sk#18, ss_net_paid#19, ss_sold_date_sk#20] +Condition : isnotnull(ss_customer_sk#18) -(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] +(21) CometBroadcastExchange +Input [3]: [ss_customer_sk#18, ss_net_paid#19, ss_sold_date_sk#20] +Arguments: [ss_customer_sk#18, ss_net_paid#19, ss_sold_date_sk#20] -(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 +(22) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#14, c_customer_id#15, c_first_name#16, c_last_name#17] +Right output [3]: [ss_customer_sk#18, ss_net_paid#19, ss_sold_date_sk#20] +Arguments: [c_customer_sk#14], [ss_customer_sk#18], Inner, BuildRight -(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] +(23) CometProject +Input [7]: [c_customer_sk#14, c_customer_id#15, c_first_name#16, c_last_name#17, ss_customer_sk#18, ss_net_paid#19, ss_sold_date_sk#20] +Arguments: [c_customer_id#15, c_first_name#16, c_last_name#17, ss_net_paid#19, ss_sold_date_sk#20], [c_customer_id#15, c_first_name#16, c_last_name#17, ss_net_paid#19, ss_sold_date_sk#20] -(25) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#23, d_year#24] +(24) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#22, d_year#23] 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 -(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 - -(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] +(25) CometFilter +Input [2]: [d_date_sk#22, d_year#23] +Condition : (((isnotnull(d_year#23) AND (d_year#23 = 2002)) AND d_year#23 IN (2001,2002)) AND isnotnull(d_date_sk#22)) + +(26) CometBroadcastExchange +Input [2]: [d_date_sk#22, d_year#23] +Arguments: [d_date_sk#22, d_year#23] + +(27) CometBroadcastHashJoin +Left output [5]: [c_customer_id#15, c_first_name#16, c_last_name#17, ss_net_paid#19, ss_sold_date_sk#20] +Right output [2]: [d_date_sk#22, d_year#23] +Arguments: [ss_sold_date_sk#20], [d_date_sk#22], Inner, BuildRight + +(28) CometProject +Input [7]: [c_customer_id#15, c_first_name#16, c_last_name#17, ss_net_paid#19, ss_sold_date_sk#20, d_date_sk#22, d_year#23] +Arguments: [c_customer_id#15, c_first_name#16, c_last_name#17, ss_net_paid#19, d_year#23], [c_customer_id#15, c_first_name#16, c_last_name#17, ss_net_paid#19, d_year#23] + +(29) CometHashAggregate +Input [5]: [c_customer_id#15, c_first_name#16, c_last_name#17, ss_net_paid#19, d_year#23] +Keys [4]: [c_customer_id#15, c_first_name#16, c_last_name#17, d_year#23] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#19))] + +(30) CometColumnarExchange +Input [5]: [c_customer_id#15, c_first_name#16, c_last_name#17, d_year#23, sum#24] +Arguments: hashpartitioning(c_customer_id#15, c_first_name#16, c_last_name#17, d_year#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(31) CometHashAggregate +Input [5]: [c_customer_id#15, c_first_name#16, c_last_name#17, d_year#23, sum#24] +Keys [4]: [c_customer_id#15, c_first_name#16, c_last_name#17, d_year#23] +Functions [1]: [sum(UnscaledValue(ss_net_paid#19))] + +(32) CometBroadcastExchange +Input [4]: [customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28] +Arguments: [customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28] + +(33) CometBroadcastHashJoin +Left output [2]: [customer_id#12, year_total#13] +Right output [4]: [customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28] +Arguments: [customer_id#12], [customer_id#25], Inner, BuildRight + +(34) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#29, c_customer_id#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_customer_id)] ReadSchema: struct -(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)) +(35) CometFilter +Input [4]: [c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] +Condition : (isnotnull(c_customer_sk#29) AND isnotnull(c_customer_id#30)) -(38) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#34, ws_net_paid#35, ws_sold_date_sk#36] +(36) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#37)] +PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#36)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(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) +(37) CometFilter +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Condition : isnotnull(ws_bill_customer_sk#33) + +(38) CometBroadcastExchange +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Arguments: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] -(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] +(39) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] +Right output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Arguments: [c_customer_sk#29], [ws_bill_customer_sk#33], Inner, BuildRight -(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 +(40) CometProject +Input [7]: [c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32, ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Arguments: [c_customer_id#30, c_first_name#31, c_last_name#32, ws_net_paid#34, ws_sold_date_sk#35], [c_customer_id#30, c_first_name#31, c_last_name#32, ws_net_paid#34, ws_sold_date_sk#35] -(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] +(41) ReusedExchange [Reuses operator id: 10] +Output [2]: [d_date_sk#37, d_year#38] -(43) ReusedExchange [Reuses operator id: 10] -Output [2]: [d_date_sk#38, d_year#39] +(42) CometBroadcastHashJoin +Left output [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, ws_net_paid#34, ws_sold_date_sk#35] +Right output [2]: [d_date_sk#37, d_year#38] +Arguments: [ws_sold_date_sk#35], [d_date_sk#37], Inner, BuildRight -(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 +(43) CometProject +Input [7]: [c_customer_id#30, c_first_name#31, c_last_name#32, ws_net_paid#34, ws_sold_date_sk#35, d_date_sk#37, d_year#38] +Arguments: [c_customer_id#30, c_first_name#31, c_last_name#32, ws_net_paid#34, d_year#38], [c_customer_id#30, c_first_name#31, c_last_name#32, ws_net_paid#34, d_year#38] -(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] +(44) CometHashAggregate +Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, ws_net_paid#34, d_year#38] +Keys [4]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#38] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#34))] + +(45) CometColumnarExchange +Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#38, sum#39] +Arguments: hashpartitioning(c_customer_id#30, c_first_name#31, c_last_name#32, d_year#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (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] - -(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 - -(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] - -(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] +Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#38, sum#39] +Keys [4]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#38] +Functions [1]: [sum(UnscaledValue(ws_net_paid#34))] + +(47) CometFilter +Input [2]: [customer_id#40, year_total#41] +Condition : (isnotnull(year_total#41) AND (year_total#41 > 0.00)) + +(48) CometBroadcastExchange +Input [2]: [customer_id#40, year_total#41] +Arguments: [customer_id#40, year_total#41] + +(49) CometBroadcastHashJoin +Left output [6]: [customer_id#12, year_total#13, customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28] +Right output [2]: [customer_id#40, year_total#41] +Arguments: [customer_id#12], [customer_id#40], Inner, BuildRight + +(50) CometProject +Input [8]: [customer_id#12, year_total#13, customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28, customer_id#40, year_total#41] +Arguments: [customer_id#12, year_total#13, customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28, year_total#41], [customer_id#12, year_total#13, customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28, year_total#41] + +(51) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#42, c_customer_id#43, c_first_name#44, c_last_name#45] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(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)) +(52) CometFilter +Input [4]: [c_customer_sk#42, c_customer_id#43, c_first_name#44, c_last_name#45] +Condition : (isnotnull(c_customer_sk#42) AND isnotnull(c_customer_id#43)) -(56) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] +(53) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_customer_sk#46, ws_net_paid#47, ws_sold_date_sk#48] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#50), dynamicpruningexpression(ws_sold_date_sk#50 IN dynamicpruning#51)] +PartitionFilters: [isnotnull(ws_sold_date_sk#48), dynamicpruningexpression(ws_sold_date_sk#48 IN dynamicpruning#49)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(57) CometFilter -Input [3]: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] -Condition : isnotnull(ws_bill_customer_sk#48) +(54) CometFilter +Input [3]: [ws_bill_customer_sk#46, ws_net_paid#47, ws_sold_date_sk#48] +Condition : isnotnull(ws_bill_customer_sk#46) -(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] +(55) CometBroadcastExchange +Input [3]: [ws_bill_customer_sk#46, ws_net_paid#47, ws_sold_date_sk#48] +Arguments: [ws_bill_customer_sk#46, ws_net_paid#47, ws_sold_date_sk#48] + +(56) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#42, c_customer_id#43, c_first_name#44, c_last_name#45] +Right output [3]: [ws_bill_customer_sk#46, ws_net_paid#47, ws_sold_date_sk#48] +Arguments: [c_customer_sk#42], [ws_bill_customer_sk#46], Inner, BuildRight + +(57) CometProject +Input [7]: [c_customer_sk#42, c_customer_id#43, c_first_name#44, c_last_name#45, ws_bill_customer_sk#46, ws_net_paid#47, ws_sold_date_sk#48] +Arguments: [c_customer_id#43, c_first_name#44, c_last_name#45, ws_net_paid#47, ws_sold_date_sk#48], [c_customer_id#43, c_first_name#44, c_last_name#45, ws_net_paid#47, ws_sold_date_sk#48] + +(58) ReusedExchange [Reuses operator id: 26] +Output [2]: [d_date_sk#50, d_year#51] (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 +Left output [5]: [c_customer_id#43, c_first_name#44, c_last_name#45, ws_net_paid#47, ws_sold_date_sk#48] +Right output [2]: [d_date_sk#50, d_year#51] +Arguments: [ws_sold_date_sk#48], [d_date_sk#50], Inner, BuildRight (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) ReusedExchange [Reuses operator id: 27] -Output [2]: [d_date_sk#52, d_year#53] - -(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) 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) 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) 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#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 : 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#55, year_total#56] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] - -(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#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 : 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#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] +Input [7]: [c_customer_id#43, c_first_name#44, c_last_name#45, ws_net_paid#47, ws_sold_date_sk#48, d_date_sk#50, d_year#51] +Arguments: [c_customer_id#43, c_first_name#44, c_last_name#45, ws_net_paid#47, d_year#51], [c_customer_id#43, c_first_name#44, c_last_name#45, ws_net_paid#47, d_year#51] + +(61) CometHashAggregate +Input [5]: [c_customer_id#43, c_first_name#44, c_last_name#45, ws_net_paid#47, d_year#51] +Keys [4]: [c_customer_id#43, c_first_name#44, c_last_name#45, d_year#51] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#47))] + +(62) CometColumnarExchange +Input [5]: [c_customer_id#43, c_first_name#44, c_last_name#45, d_year#51, sum#52] +Arguments: hashpartitioning(c_customer_id#43, c_first_name#44, c_last_name#45, d_year#51, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(63) CometHashAggregate +Input [5]: [c_customer_id#43, c_first_name#44, c_last_name#45, d_year#51, sum#52] +Keys [4]: [c_customer_id#43, c_first_name#44, c_last_name#45, d_year#51] +Functions [1]: [sum(UnscaledValue(ws_net_paid#47))] + +(64) CometBroadcastExchange +Input [2]: [customer_id#53, year_total#54] +Arguments: [customer_id#53, year_total#54] + +(65) CometBroadcastHashJoin +Left output [7]: [customer_id#12, year_total#13, customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28, year_total#41] +Right output [2]: [customer_id#53, year_total#54] +Arguments: [customer_id#12], [customer_id#53], Inner, (CASE WHEN (year_total#41 > 0.00) THEN (year_total#54 / year_total#41) END > CASE WHEN (year_total#13 > 0.00) THEN (year_total#28 / year_total#13) END), BuildRight + +(66) CometProject +Input [9]: [customer_id#12, year_total#13, customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28, year_total#41, customer_id#53, year_total#54] +Arguments: [customer_id#25, customer_first_name#26, customer_last_name#27], [customer_id#25, customer_first_name#26, customer_last_name#27] + +(67) CometTakeOrderedAndProject +Input [3]: [customer_id#25, customer_first_name#26, customer_last_name#27] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_first_name#26 ASC NULLS FIRST,customer_id#25 ASC NULLS FIRST,customer_last_name#27 ASC NULLS FIRST], output=[customer_id#25,customer_first_name#26,customer_last_name#27]), [customer_id#25, customer_first_name#26, customer_last_name#27], 100, [customer_first_name#26 ASC NULLS FIRST, customer_id#25 ASC NULLS FIRST, customer_last_name#27 ASC NULLS FIRST], [customer_id#25, customer_first_name#26, customer_last_name#27] + +(68) ColumnarToRow [codegen id : 1] +Input [3]: [customer_id#25, customer_first_name#26, customer_last_name#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (75) -+- * ColumnarToRow (74) - +- CometFilter (73) - +- CometScan parquet spark_catalog.default.date_dim (72) +BroadcastExchange (72) ++- * ColumnarToRow (71) + +- CometFilter (70) + +- CometScan parquet spark_catalog.default.date_dim (69) -(72) Scan parquet spark_catalog.default.date_dim +(69) 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 -(73) CometFilter +(70) 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)) -(74) ColumnarToRow [codegen id : 1] +(71) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_year#10] -(75) BroadcastExchange +(72) BroadcastExchange Input [2]: [d_date_sk#9, d_year#10] -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=5] -Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#21 IN dynamicpruning#22 -BroadcastExchange (79) -+- * ColumnarToRow (78) - +- CometFilter (77) - +- CometScan parquet spark_catalog.default.date_dim (76) +Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#21 +BroadcastExchange (76) ++- * ColumnarToRow (75) + +- CometFilter (74) + +- CometScan parquet spark_catalog.default.date_dim (73) -(76) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#23, d_year#24] +(73) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#22, d_year#23] 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#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)) +(74) CometFilter +Input [2]: [d_date_sk#22, d_year#23] +Condition : (((isnotnull(d_year#23) AND (d_year#23 = 2002)) AND d_year#23 IN (2001,2002)) AND isnotnull(d_date_sk#22)) -(78) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#23, d_year#24] +(75) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#22, d_year#23] -(79) BroadcastExchange -Input [2]: [d_date_sk#23, d_year#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] +(76) BroadcastExchange +Input [2]: [d_date_sk#22, d_year#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -Subquery:3 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#8 -Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#50 IN dynamicpruning#22 +Subquery:4 Hosting operator id = 53 Hosting Expression = ws_sold_date_sk#48 IN dynamicpruning#21 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/simplified.txt index e5cf23f50..0b288df94 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/simplified.txt @@ -1,106 +1,86 @@ -TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] - 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] - BroadcastHashJoin [customer_id,customer_id] - BroadcastHashJoin [customer_id,customer_id] - Filter [year_total] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_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_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - 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,sum,ss_net_paid] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - 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] #11 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 - CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_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 #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] #14 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 - CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_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 +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] + CometProject [customer_id,customer_first_name,customer_last_name] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total,customer_id,year_total] + CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,customer_id,year_total] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total] + CometFilter [customer_id,year_total] + CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,d_year,sum,sum(UnscaledValue(ss_net_paid))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_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_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,year_total] #5 + CometHashAggregate [customer_id,customer_first_name,customer_last_name,year_total,c_customer_id,c_first_name,c_last_name,d_year,sum,sum(UnscaledValue(ss_net_paid))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #6 + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [customer_id,year_total] #10 + CometFilter [customer_id,year_total] + CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,d_year,sum,sum(UnscaledValue(ws_net_paid))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #11 + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_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 + CometBroadcastExchange [customer_id,year_total] #13 + CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,d_year,sum,sum(UnscaledValue(ws_net_paid))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #14 + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_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-spark3_5/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/explain.txt index 564e40928..2aeff8de1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/explain.txt @@ -1,136 +1,129 @@ == Physical Plan == -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) +TakeOrderedAndProject (125) ++- * Project (124) + +- * SortMergeJoin Inner (123) + :- * ColumnarToRow (67) + : +- CometSort (66) + : +- CometColumnarExchange (65) + : +- CometFilter (64) + : +- CometHashAggregate (63) + : +- CometColumnarExchange (62) + : +- CometHashAggregate (61) + : +- CometHashAggregate (60) + : +- CometColumnarExchange (59) + : +- CometHashAggregate (58) + : +- CometUnion (57) + : :- CometProject (22) + : : +- CometSortMergeJoin (21) + : : :- CometSort (15) + : : : +- CometColumnarExchange (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) + : : +- CometSort (20) + : : +- CometColumnarExchange (19) + : : +- CometProject (18) + : : +- CometFilter (17) + : : +- CometScan parquet spark_catalog.default.catalog_returns (16) + : :- CometProject (39) + : : +- CometSortMergeJoin (38) + : : :- CometSort (32) + : : : +- CometColumnarExchange (31) + : : : +- CometProject (30) + : : : +- CometBroadcastHashJoin (29) + : : : :- CometProject (27) + : : : : +- CometBroadcastHashJoin (26) + : : : : :- CometFilter (24) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (23) + : : : : +- ReusedExchange (25) + : : : +- ReusedExchange (28) + : : +- CometSort (37) + : : +- CometColumnarExchange (36) + : : +- CometProject (35) + : : +- CometFilter (34) + : : +- CometScan parquet spark_catalog.default.store_returns (33) + : +- CometProject (56) + : +- CometSortMergeJoin (55) + : :- CometSort (49) + : : +- CometColumnarExchange (48) + : : +- CometProject (47) + : : +- CometBroadcastHashJoin (46) + : : :- CometProject (44) + : : : +- CometBroadcastHashJoin (43) + : : : :- CometFilter (41) + : : : : +- CometScan parquet spark_catalog.default.web_sales (40) + : : : +- ReusedExchange (42) + : : +- ReusedExchange (45) + : +- CometSort (54) + : +- CometColumnarExchange (53) + : +- CometProject (52) + : +- CometFilter (51) + : +- CometScan parquet spark_catalog.default.web_returns (50) + +- * ColumnarToRow (122) + +- CometSort (121) + +- CometColumnarExchange (120) + +- CometFilter (119) + +- CometHashAggregate (118) + +- CometColumnarExchange (117) + +- CometHashAggregate (116) + +- CometHashAggregate (115) + +- CometColumnarExchange (114) + +- CometHashAggregate (113) + +- CometUnion (112) + :- CometProject (83) + : +- CometSortMergeJoin (82) + : :- CometSort (79) + : : +- CometColumnarExchange (78) + : : +- CometProject (77) + : : +- CometBroadcastHashJoin (76) + : : :- CometProject (72) + : : : +- CometBroadcastHashJoin (71) + : : : :- CometFilter (69) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (68) + : : : +- ReusedExchange (70) + : : +- CometBroadcastExchange (75) + : : +- CometFilter (74) + : : +- CometScan parquet spark_catalog.default.date_dim (73) + : +- CometSort (81) + : +- ReusedExchange (80) + :- CometProject (97) + : +- CometSortMergeJoin (96) + : :- CometSort (93) + : : +- CometColumnarExchange (92) + : : +- CometProject (91) + : : +- CometBroadcastHashJoin (90) + : : :- CometProject (88) + : : : +- CometBroadcastHashJoin (87) + : : : :- CometFilter (85) + : : : : +- CometScan parquet spark_catalog.default.store_sales (84) + : : : +- ReusedExchange (86) + : : +- ReusedExchange (89) + : +- CometSort (95) + : +- ReusedExchange (94) + +- CometProject (111) + +- CometSortMergeJoin (110) + :- CometSort (107) + : +- CometColumnarExchange (106) + : +- CometProject (105) + : +- CometBroadcastHashJoin (104) + : :- CometProject (102) + : : +- CometBroadcastHashJoin (101) + : : :- CometFilter (99) + : : : +- CometScan parquet spark_catalog.default.web_sales (98) + : : +- ReusedExchange (100) + : +- ReusedExchange (103) + +- CometSort (109) + +- ReusedExchange (108) (1) Scan parquet spark_catalog.default.catalog_sales @@ -197,54 +190,47 @@ Arguments: [cs_sold_date_sk#5], [d_date_sk#13], Inner, BuildRight 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] +(14) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(15) Exchange +(15) CometSort 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=1] +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_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST] -(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 - -(17) Scan parquet spark_catalog.default.catalog_returns +(16) 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 -(18) CometFilter +(17) 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)) -(19) CometProject +(18) 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] -(20) ColumnarToRow [codegen id : 3] -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] - -(21) Exchange +(19) CometColumnarExchange 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=2] +Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(22) Sort [codegen id : 4] +(20) CometSort 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 +Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST] -(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 +(21) CometSortMergeJoin +Left 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] +Right output [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] +Arguments: [cs_order_number#2, cs_item_sk#1], [cr_order_number#16, cr_item_sk#15], LeftOuter -(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] +(22) CometProject 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] +Arguments: [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], [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] -(25) Scan parquet spark_catalog.default.store_sales +(23) 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 [] @@ -252,82 +238,75 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(26) CometFilter +(24) 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) -(27) ReusedExchange [Reuses operator id: 6] +(25) 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) CometBroadcastHashJoin +(26) 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) CometProject +(27) 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: 11] +(28) ReusedExchange [Reuses operator id: 11] Output [2]: [d_date_sk#33, d_year#34] -(31) CometBroadcastHashJoin +(29) 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) CometProject +(30) 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] +(31) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=3] -(34) Exchange +(32) CometSort 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=3] +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_ticket_number#23 ASC NULLS FIRST, ss_item_sk#22 ASC NULLS FIRST] -(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 - -(36) Scan parquet spark_catalog.default.store_returns +(33) 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 -(37) CometFilter +(34) 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)) -(38) CometProject +(35) 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] -(39) ColumnarToRow [codegen id : 8] +(36) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=4] -(40) Exchange +(37) CometSort 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=4] +Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST] -(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 +(38) CometSortMergeJoin +Left 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] +Right output [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] +Arguments: [ss_ticket_number#23, ss_item_sk#22], [sr_ticket_number#36, sr_item_sk#35], LeftOuter -(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 - -(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] +(39) CometProject 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] +Arguments: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41], [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] -(44) Scan parquet spark_catalog.default.web_sales +(40) 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 [] @@ -335,457 +314,434 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(45) CometFilter +(41) 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) -(46) ReusedExchange [Reuses operator id: 6] +(42) 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) CometBroadcastHashJoin +(43) 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) CometProject +(44) 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: 11] +(45) ReusedExchange [Reuses operator id: 11] Output [2]: [d_date_sk#53, d_year#54] -(50) CometBroadcastHashJoin +(46) 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) CometProject +(47) 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) 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] - -(53) Exchange +(48) CometColumnarExchange 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] +Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(54) Sort [codegen id : 12] +(49) CometSort 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 +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_order_number#43 ASC NULLS FIRST, ws_item_sk#42 ASC NULLS FIRST] -(55) Scan parquet spark_catalog.default.web_returns +(50) 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 -(56) CometFilter +(51) 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)) -(57) CometProject +(52) 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] -(58) ColumnarToRow [codegen id : 13] +(53) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=6] -(59) Exchange +(54) CometSort 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=6] +Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST] -(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 - -(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 +(55) CometSortMergeJoin +Left 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] +Right output [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] +Arguments: [ws_order_number#43, ws_item_sk#42], [wr_order_number#56, wr_item_sk#55], LeftOuter -(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] +(56) CometProject 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] +Arguments: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61], [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] -(63) Union +(57) CometUnion +Child 0 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] +Child 1 Input [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41] +Child 2 Input [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61] -(64) HashAggregate [codegen id : 16] +(58) CometHashAggregate 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] -(65) Exchange +(59) CometColumnarExchange 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=7] +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, CometColumnarShuffle, [plan_id=7] -(66) HashAggregate [codegen id : 17] +(60) CometHashAggregate 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] -(67) HashAggregate [codegen id : 17] +(61) CometHashAggregate 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] -(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=8] +(62) CometColumnarExchange +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] +Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(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] +(63) CometHashAggregate +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] 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] -(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) +(64) CometFilter +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] +Condition : isnotnull(sales_cnt#64) + +(65) CometColumnarExchange +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(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=9] +(66) CometSort +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] +Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65], [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] -(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 +(67) ColumnarToRow [codegen id : 1] +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -(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] +(68) Scan parquet spark_catalog.default.catalog_sales +Output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#74), dynamicpruningexpression(cs_sold_date_sk#74 IN dynamicpruning#75)] +PartitionFilters: [isnotnull(cs_sold_date_sk#70), dynamicpruningexpression(cs_sold_date_sk#70 IN dynamicpruning#71)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(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) +(69) CometFilter +Input [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] +Condition : isnotnull(cs_item_sk#66) -(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] +(70) ReusedExchange [Reuses operator id: 6] +Output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -(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 +(71) CometBroadcastHashJoin +Left output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] +Right output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] +Arguments: [cs_item_sk#66], [i_item_sk#72], Inner, BuildRight -(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] +(72) CometProject +Input [10]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] +Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -(78) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#81, d_year#82] +(73) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#77, d_year#78] 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) 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) CometBroadcastExchange -Input [2]: [d_date_sk#81, d_year#82] -Arguments: [d_date_sk#81, d_year#82] +(74) CometFilter +Input [2]: [d_date_sk#77, d_year#78] +Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) -(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 +(75) CometBroadcastExchange +Input [2]: [d_date_sk#77, d_year#78] +Arguments: [d_date_sk#77, d_year#78] -(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] +(76) CometBroadcastHashJoin +Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] +Right output [2]: [d_date_sk#77, d_year#78] +Arguments: [cs_sold_date_sk#70], [d_date_sk#77], Inner, BuildRight -(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] +(77) CometProject +Input [11]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_date_sk#77, d_year#78] +Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -(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=10] +(78) CometColumnarExchange +Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] +Arguments: hashpartitioning(cs_order_number#67, cs_item_sk#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(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 +(79) CometSort +Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] +Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_order_number#67 ASC NULLS FIRST, cs_item_sk#66 ASC NULLS FIRST] -(86) ReusedExchange [Reuses operator id: 21] -Output [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] +(80) ReusedExchange [Reuses operator id: 19] +Output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -(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 +(81) CometSort +Input [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] +Arguments: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82], [cr_order_number#80 ASC NULLS FIRST, cr_item_sk#79 ASC NULLS FIRST] -(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 +(82) CometSortMergeJoin +Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] +Right output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] +Arguments: [cs_order_number#67, cs_item_sk#66], [cr_order_number#80, cr_item_sk#79], LeftOuter -(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] +(83) CometProject +Input [13]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78, cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] +Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21], [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, (cs_quantity#68 - coalesce(cr_return_quantity#81, 0)) AS sales_cnt#20, (cs_ext_sales_price#69 - coalesce(cr_return_amount#82, 0.00)) AS sales_amt#21] -(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] +(84) Scan parquet spark_catalog.default.store_sales +Output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#91), dynamicpruningexpression(ss_sold_date_sk#91 IN dynamicpruning#92)] +PartitionFilters: [isnotnull(ss_sold_date_sk#87), dynamicpruningexpression(ss_sold_date_sk#87 IN dynamicpruning#88)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(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) - -(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] +(85) CometFilter +Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] +Condition : isnotnull(ss_item_sk#83) -(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 +(86) ReusedExchange [Reuses operator id: 6] +Output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -(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] +(87) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] +Right output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] +Arguments: [ss_item_sk#83], [i_item_sk#89], Inner, BuildRight -(95) ReusedExchange [Reuses operator id: 80] -Output [2]: [d_date_sk#98, d_year#99] +(88) CometProject +Input [10]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] +Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93], [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -(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 +(89) ReusedExchange [Reuses operator id: 75] +Output [2]: [d_date_sk#94, d_year#95] -(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] +(90) CometBroadcastHashJoin +Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] +Right output [2]: [d_date_sk#94, d_year#95] +Arguments: [ss_sold_date_sk#87], [d_date_sk#94], Inner, BuildRight -(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] +(91) CometProject +Input [11]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_date_sk#94, d_year#95] +Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -(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] +(92) CometColumnarExchange +Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] +Arguments: hashpartitioning(ss_ticket_number#84, ss_item_sk#83, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [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 +(93) CometSort +Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] +Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_ticket_number#84 ASC NULLS FIRST, ss_item_sk#83 ASC NULLS FIRST] -(101) ReusedExchange [Reuses operator id: 40] -Output [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] +(94) ReusedExchange [Reuses operator id: 36] +Output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -(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 +(95) CometSort +Input [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] +Arguments: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99], [sr_ticket_number#97 ASC NULLS FIRST, sr_item_sk#96 ASC NULLS FIRST] -(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 +(96) CometSortMergeJoin +Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] +Right output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] +Arguments: [ss_ticket_number#84, ss_item_sk#83], [sr_ticket_number#97, sr_item_sk#96], LeftOuter -(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] +(97) CometProject +Input [13]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95, sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] +Arguments: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41], [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, (ss_quantity#85 - coalesce(sr_return_quantity#98, 0)) AS sales_cnt#40, (ss_ext_sales_price#86 - coalesce(sr_return_amt#99, 0.00)) AS sales_amt#41] -(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] +(98) Scan parquet spark_catalog.default.web_sales +Output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#108), dynamicpruningexpression(ws_sold_date_sk#108 IN dynamicpruning#109)] +PartitionFilters: [isnotnull(ws_sold_date_sk#104), dynamicpruningexpression(ws_sold_date_sk#104 IN dynamicpruning#105)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(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) - -(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] +(99) CometFilter +Input [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] +Condition : isnotnull(ws_item_sk#100) -(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 +(100) ReusedExchange [Reuses operator id: 6] +Output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -(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] +(101) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] +Right output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] +Arguments: [ws_item_sk#100], [i_item_sk#106], Inner, BuildRight -(110) ReusedExchange [Reuses operator id: 80] -Output [2]: [d_date_sk#115, d_year#116] +(102) CometProject +Input [10]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] +Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110], [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -(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 +(103) ReusedExchange [Reuses operator id: 75] +Output [2]: [d_date_sk#111, d_year#112] -(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] +(104) CometBroadcastHashJoin +Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] +Right output [2]: [d_date_sk#111, d_year#112] +Arguments: [ws_sold_date_sk#104], [d_date_sk#111], Inner, BuildRight -(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] +(105) CometProject +Input [11]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_date_sk#111, d_year#112] +Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -(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=12] +(106) CometColumnarExchange +Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] +Arguments: hashpartitioning(ws_order_number#101, ws_item_sk#100, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(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 +(107) CometSort +Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] +Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_order_number#101 ASC NULLS FIRST, ws_item_sk#100 ASC NULLS FIRST] -(116) ReusedExchange [Reuses operator id: 59] -Output [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] +(108) ReusedExchange [Reuses operator id: 53] +Output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -(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 +(109) CometSort +Input [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] +Arguments: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116], [wr_order_number#114 ASC NULLS FIRST, wr_item_sk#113 ASC NULLS FIRST] -(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 +(110) CometSortMergeJoin +Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] +Right output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] +Arguments: [ws_order_number#101, ws_item_sk#100], [wr_order_number#114, wr_item_sk#113], LeftOuter -(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] +(111) CometProject +Input [13]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112, wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] +Arguments: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61], [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, (ws_quantity#102 - coalesce(wr_return_quantity#115, 0)) AS sales_cnt#60, (ws_ext_sales_price#103 - coalesce(wr_return_amt#116, 0.00)) AS sales_amt#61] -(120) Union +(112) CometUnion +Child 0 Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] +Child 1 Input [7]: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41] +Child 2 Input [7]: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61] -(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] +(113) CometHashAggregate +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] +Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 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] -(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=13] +(114) CometColumnarExchange +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] +Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(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] +(115) CometHashAggregate +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] +Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 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] -(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] +(116) CometHashAggregate +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] +Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] 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] -(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=14] +(117) CometColumnarExchange +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] +Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] -(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] +(118) CometHashAggregate +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] +Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] 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] -(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) +(119) CometFilter +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] +Condition : isnotnull(sales_cnt#118) + +(120) CometColumnarExchange +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] +Arguments: hashpartitioning(i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] -(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=15] +(121) CometSort +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] +Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119], [i_brand_id#73 ASC NULLS FIRST, i_class_id#74 ASC NULLS FIRST, i_category_id#75 ASC NULLS FIRST, i_manufact_id#76 ASC NULLS FIRST] -(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 +(122) ColumnarToRow [codegen id : 2] +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -(130) SortMergeJoin [codegen id : 39] +(123) SortMergeJoin [codegen id : 3] 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] +Right keys [4]: [i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] Join type: Inner -Join condition: ((cast(sales_cnt#68 as decimal(17,2)) / cast(sales_cnt#123 as decimal(17,2))) < 0.90000000000000000000) +Join condition: ((cast(sales_cnt#64 as decimal(17,2)) / cast(sales_cnt#118 as decimal(17,2))) < 0.90000000000000000000) -(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] +(124) Project [codegen id : 3] +Output [10]: [d_year#78 AS prev_year#120, d_year#14 AS year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#118 AS prev_yr_cnt#122, sales_cnt#64 AS curr_yr_cnt#123, (sales_cnt#64 - sales_cnt#118) AS sales_cnt_diff#124, (sales_amt#65 - sales_amt#119) AS sales_amt_diff#125] +Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65, d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -(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] +(125) TakeOrderedAndProject +Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] +Arguments: 100, [sales_cnt_diff#124 ASC NULLS FIRST, sales_amt_diff#125 ASC NULLS FIRST], [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (136) -+- * ColumnarToRow (135) - +- CometFilter (134) - +- CometScan parquet spark_catalog.default.date_dim (133) +BroadcastExchange (129) ++- * ColumnarToRow (128) + +- CometFilter (127) + +- CometScan parquet spark_catalog.default.date_dim (126) -(133) Scan parquet spark_catalog.default.date_dim +(126) 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 -(134) CometFilter +(127) 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)) -(135) ColumnarToRow [codegen id : 1] +(128) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] -(136) BroadcastExchange +(129) BroadcastExchange Input [2]: [d_date_sk#13, d_year#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 -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) +Subquery:4 Hosting operator id = 68 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 +BroadcastExchange (133) ++- * ColumnarToRow (132) + +- CometFilter (131) + +- CometScan parquet spark_catalog.default.date_dim (130) -(137) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#81, d_year#82] +(130) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#77, d_year#78] 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 -(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)) +(131) CometFilter +Input [2]: [d_date_sk#77, d_year#78] +Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) -(139) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#81, d_year#82] +(132) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#77, d_year#78] -(140) BroadcastExchange -Input [2]: [d_date_sk#81, d_year#82] +(133) BroadcastExchange +Input [2]: [d_date_sk#77, d_year#78] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] -Subquery:5 Hosting operator id = 90 Hosting Expression = ss_sold_date_sk#91 IN dynamicpruning#75 +Subquery:5 Hosting operator id = 84 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#71 -Subquery:6 Hosting operator id = 105 Hosting Expression = ws_sold_date_sk#108 IN dynamicpruning#75 +Subquery:6 Hosting operator id = 98 Hosting Expression = ws_sold_date_sk#104 IN dynamicpruning#71 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/simplified.txt index 2aecc9e98..9b187bca1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/simplified.txt @@ -1,229 +1,150 @@ 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 (39) + WholeStageCodegen (3) 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 (19) - Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - 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 (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 (16) - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - InputAdapter - Union - 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 (2) - Sort [cs_order_number,cs_item_sk] - InputAdapter - Exchange [cs_order_number,cs_item_sk] #4 - 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_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,d_date_sk,d_year] - 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,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_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 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,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 [d_date_sk,d_year] #7 - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - WholeStageCodegen (4) - Sort [cr_order_number,cr_item_sk] - InputAdapter - 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_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_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 (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 (7) - Sort [ss_ticket_number,ss_item_sk] - 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_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,d_date_sk,d_year] - 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,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_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 - 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 (9) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - 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_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_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 (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 (12) - Sort [ws_order_number,ws_item_sk] - 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_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,d_date_sk,d_year] - 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,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_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 - 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 (14) - Sort [wr_order_number,wr_item_sk] - InputAdapter - 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_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 + CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum,sum,sum(sales_cnt),sum(UnscaledValue(sales_amt))] + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum,sales_cnt,sales_amt] + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometSort [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] + CometColumnarExchange [cs_order_number,cs_item_sk] #4 + 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_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,d_date_sk,d_year] + 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,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_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 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,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 [d_date_sk,d_year] #7 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometColumnarExchange [cr_order_number,cr_item_sk] #8 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometSort [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] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #9 + 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_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,d_date_sk,d_year] + 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,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_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 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #7 + CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometColumnarExchange [sr_ticket_number,sr_item_sk] #10 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometSort [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] + CometColumnarExchange [ws_order_number,ws_item_sk] #11 + 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_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,d_date_sk,d_year] + 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,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_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 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #7 + CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometColumnarExchange [wr_order_number,wr_item_sk] #12 + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_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 (38) - Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] + WholeStageCodegen (2) + ColumnarToRow InputAdapter - 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] #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] #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 (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 (21) - Sort [cs_order_number,cs_item_sk] - 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_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,d_date_sk,d_year] - 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,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_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 #17 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometBroadcastExchange [d_date_sk,d_year] #18 - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - WholeStageCodegen (23) - Sort [cr_order_number,cr_item_sk] - InputAdapter - 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 (26) - Sort [ss_ticket_number,ss_item_sk] - 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_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,d_date_sk,d_year] - 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,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_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 - 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 (28) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - 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 (31) - Sort [ws_order_number,ws_item_sk] - 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_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,d_date_sk,d_year] - 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,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_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 - 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 (33) - Sort [wr_order_number,wr_item_sk] - InputAdapter - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #12 + CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 + CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum,sum,sum(sales_cnt),sum(UnscaledValue(sales_amt))] + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum,sales_cnt,sales_amt] + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometSort [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] + CometColumnarExchange [cs_order_number,cs_item_sk] #16 + 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_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,d_date_sk,d_year] + 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,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_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 #17 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + CometBroadcastExchange [d_date_sk,d_year] #18 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #8 + CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometSort [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] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #19 + 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_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,d_date_sk,d_year] + 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,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_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 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #18 + CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #10 + CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometSort [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] + CometColumnarExchange [ws_order_number,ws_item_sk] #20 + 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_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,d_date_sk,d_year] + 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,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_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 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #18 + CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + 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-spark3_5/q77a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/explain.txt index 7b4306581..a2adcb0b5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/explain.txt @@ -1,103 +1,111 @@ == Physical Plan == -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) +* ColumnarToRow (107) ++- CometTakeOrderedAndProject (106) + +- CometHashAggregate (105) + +- CometColumnarExchange (104) + +- RowToColumnar (103) + +- * HashAggregate (102) + +- Union (101) + :- * HashAggregate (84) + : +- * ColumnarToRow (83) + : +- CometColumnarExchange (82) + : +- RowToColumnar (81) + : +- * HashAggregate (80) + : +- Union (79) + : :- * ColumnarToRow (31) + : : +- CometProject (30) + : : +- CometBroadcastHashJoin (29) + : : :- CometHashAggregate (16) + : : : +- CometColumnarExchange (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) + : : +- CometBroadcastExchange (28) + : : +- CometHashAggregate (27) + : : +- CometColumnarExchange (26) + : : +- CometHashAggregate (25) + : : +- CometProject (24) + : : +- CometBroadcastHashJoin (23) + : : :- CometProject (21) + : : : +- CometBroadcastHashJoin (20) + : : : :- CometFilter (18) + : : : : +- CometScan parquet spark_catalog.default.store_returns (17) + : : : +- ReusedExchange (19) + : : +- ReusedExchange (22) + : :- * Project (50) + : : +- * BroadcastNestedLoopJoin Inner BuildLeft (49) + : : :- BroadcastExchange (40) + : : : +- * ColumnarToRow (39) + : : : +- CometHashAggregate (38) + : : : +- CometColumnarExchange (37) + : : : +- CometHashAggregate (36) + : : : +- CometProject (35) + : : : +- CometBroadcastHashJoin (34) + : : : :- CometScan parquet spark_catalog.default.catalog_sales (32) + : : : +- ReusedExchange (33) + : : +- * ColumnarToRow (48) + : : +- CometHashAggregate (47) + : : +- CometColumnarExchange (46) + : : +- CometHashAggregate (45) + : : +- CometProject (44) + : : +- CometBroadcastHashJoin (43) + : : :- CometScan parquet spark_catalog.default.catalog_returns (41) + : : +- ReusedExchange (42) + : +- * ColumnarToRow (78) + : +- CometProject (77) + : +- CometBroadcastHashJoin (76) + : :- CometHashAggregate (63) + : : +- CometColumnarExchange (62) + : : +- CometHashAggregate (61) + : : +- CometProject (60) + : : +- CometBroadcastHashJoin (59) + : : :- CometProject (55) + : : : +- CometBroadcastHashJoin (54) + : : : :- CometFilter (52) + : : : : +- CometScan parquet spark_catalog.default.web_sales (51) + : : : +- ReusedExchange (53) + : : +- CometBroadcastExchange (58) + : : +- CometFilter (57) + : : +- CometScan parquet spark_catalog.default.web_page (56) + : +- CometBroadcastExchange (75) + : +- CometHashAggregate (74) + : +- CometColumnarExchange (73) + : +- CometHashAggregate (72) + : +- CometProject (71) + : +- CometBroadcastHashJoin (70) + : :- CometProject (68) + : : +- CometBroadcastHashJoin (67) + : : :- CometFilter (65) + : : : +- CometScan parquet spark_catalog.default.web_returns (64) + : : +- ReusedExchange (66) + : +- ReusedExchange (69) + :- * HashAggregate (92) + : +- * ColumnarToRow (91) + : +- CometColumnarExchange (90) + : +- RowToColumnar (89) + : +- * HashAggregate (88) + : +- * HashAggregate (87) + : +- * ColumnarToRow (86) + : +- ReusedExchange (85) + +- * HashAggregate (100) + +- * ColumnarToRow (99) + +- CometColumnarExchange (98) + +- RowToColumnar (97) + +- * HashAggregate (96) + +- * HashAggregate (95) + +- * ColumnarToRow (94) + +- ReusedExchange (93) (1) Scan parquet spark_catalog.default.store_sales @@ -169,453 +177,461 @@ 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))] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarExchange Input [3]: [s_store_sk#8, sum#9, sum#10] +Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(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] +(16) CometHashAggregate 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))#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] -(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] +(17) Scan parquet spark_catalog.default.store_returns +Output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#18), dynamicpruningexpression(sr_returned_date_sk#18 IN dynamicpruning#19)] +PartitionFilters: [isnotnull(sr_returned_date_sk#14), dynamicpruningexpression(sr_returned_date_sk#14 IN dynamicpruning#15)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(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) - -(20) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#20] - -(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 - -(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] - -(23) ReusedExchange [Reuses operator id: 11] -Output [1]: [s_store_sk#21] - -(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 - -(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] - -(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 +(18) CometFilter +Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] +Condition : isnotnull(sr_store_sk#11) + +(19) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#16] + +(20) CometBroadcastHashJoin +Left output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] +Right output [1]: [d_date_sk#16] +Arguments: [sr_returned_date_sk#14], [d_date_sk#16], Inner, BuildRight + +(21) CometProject +Input [5]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14, d_date_sk#16] +Arguments: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13], [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] + +(22) ReusedExchange [Reuses operator id: 11] +Output [1]: [s_store_sk#17] + +(23) CometBroadcastHashJoin +Left output [3]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] +Right output [1]: [s_store_sk#17] +Arguments: [sr_store_sk#11], [s_store_sk#17], Inner, BuildRight + +(24) CometProject +Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, s_store_sk#17] +Arguments: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17], [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] + +(25) CometHashAggregate +Input [3]: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] +Keys [1]: [s_store_sk#17] +Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#12)), partial_sum(UnscaledValue(sr_net_loss#13))] + +(26) CometColumnarExchange +Input [3]: [s_store_sk#17, sum#18, sum#19] +Arguments: hashpartitioning(s_store_sk#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(27) CometHashAggregate +Input [3]: [s_store_sk#17, sum#18, sum#19] +Keys [1]: [s_store_sk#17] +Functions [2]: [sum(UnscaledValue(sr_return_amt#12)), sum(UnscaledValue(sr_net_loss#13))] + +(28) CometBroadcastExchange +Input [3]: [s_store_sk#17, returns#20, profit_loss#21] +Arguments: [s_store_sk#17, returns#20, profit_loss#21] + +(29) CometBroadcastHashJoin +Left output [3]: [s_store_sk#8, sales#22, profit#23] +Right output [3]: [s_store_sk#17, returns#20, profit_loss#21] +Arguments: [s_store_sk#8], [s_store_sk#17], LeftOuter, BuildRight -(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] +(30) CometProject +Input [6]: [s_store_sk#8, sales#22, profit#23, s_store_sk#17, returns#20, profit_loss#21] +Arguments: [channel#24, id#25, sales#22, returns#26, profit#27], [store channel AS channel#24, s_store_sk#8 AS id#25, sales#22, coalesce(returns#20, 0.00) AS returns#26, (profit#23 - coalesce(profit_loss#21, 0.00)) AS profit#27] -(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] +(31) ColumnarToRow [codegen id : 1] +Input [5]: [channel#24, id#25, sales#22, returns#26, profit#27] + +(32) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#35), dynamicpruningexpression(cs_sold_date_sk#35 IN dynamicpruning#36)] +PartitionFilters: [isnotnull(cs_sold_date_sk#31), dynamicpruningexpression(cs_sold_date_sk#31 IN dynamicpruning#32)] ReadSchema: struct -(34) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#37] +(33) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#33] -(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) CometBroadcastHashJoin +Left output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] +Right output [1]: [d_date_sk#33] +Arguments: [cs_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight -(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) CometProject +Input [5]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31, d_date_sk#33] +Arguments: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30], [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] -(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))] +(36) CometHashAggregate +Input [3]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] +Keys [1]: [cs_call_center_sk#28] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#29)), partial_sum(UnscaledValue(cs_net_profit#30))] -(38) ColumnarToRow [codegen id : 5] -Input [3]: [cs_call_center_sk#32, sum#38, sum#39] +(37) CometColumnarExchange +Input [3]: [cs_call_center_sk#28, sum#34, sum#35] +Arguments: hashpartitioning(cs_call_center_sk#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(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] +(38) CometHashAggregate +Input [3]: [cs_call_center_sk#28, sum#34, sum#35] +Keys [1]: [cs_call_center_sk#28] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#29)), sum(UnscaledValue(cs_net_profit#30))] -(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] +(39) ColumnarToRow [codegen id : 2] +Input [3]: [cs_call_center_sk#28, sales#36, profit#37] -(41) BroadcastExchange -Input [3]: [cs_call_center_sk#32, sales#42, profit#43] -Arguments: IdentityBroadcastMode, [plan_id=5] +(40) BroadcastExchange +Input [3]: [cs_call_center_sk#28, sales#36, profit#37] +Arguments: IdentityBroadcastMode, [plan_id=4] -(42) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#44, cr_net_loss#45, cr_returned_date_sk#46] +(41) Scan parquet spark_catalog.default.catalog_returns +Output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#46), dynamicpruningexpression(cr_returned_date_sk#46 IN dynamicpruning#47)] +PartitionFilters: [isnotnull(cr_returned_date_sk#40), dynamicpruningexpression(cr_returned_date_sk#40 IN dynamicpruning#41)] ReadSchema: struct -(43) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#48] +(42) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#42] -(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) CometBroadcastHashJoin +Left output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] +Right output [1]: [d_date_sk#42] +Arguments: [cr_returned_date_sk#40], [d_date_sk#42], Inner, BuildRight -(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) CometProject +Input [4]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40, d_date_sk#42] +Arguments: [cr_return_amount#38, cr_net_loss#39], [cr_return_amount#38, cr_net_loss#39] -(46) CometHashAggregate -Input [2]: [cr_return_amount#44, cr_net_loss#45] +(45) CometHashAggregate +Input [2]: [cr_return_amount#38, cr_net_loss#39] Keys: [] -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] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#38)), partial_sum(UnscaledValue(cr_net_loss#39))] -(48) Exchange -Input [2]: [sum#49, sum#50] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +(46) CometColumnarExchange +Input [2]: [sum#43, sum#44] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(49) HashAggregate -Input [2]: [sum#49, sum#50] +(47) CometHashAggregate +Input [2]: [sum#43, sum#44] Keys: [] -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] +Functions [2]: [sum(UnscaledValue(cr_return_amount#38)), sum(UnscaledValue(cr_net_loss#39))] -(50) BroadcastNestedLoopJoin [codegen id : 8] +(48) ColumnarToRow +Input [2]: [returns#45, profit_loss#46] + +(49) BroadcastNestedLoopJoin [codegen id : 3] Join type: Inner Join condition: None -(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) Project [codegen id : 3] +Output [5]: [catalog channel AS channel#47, cs_call_center_sk#28 AS id#48, sales#36, returns#45, (profit#37 - profit_loss#46) AS profit#49] +Input [5]: [cs_call_center_sk#28, sales#36, profit#37, returns#45, profit_loss#46] -(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] +(51) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#61), dynamicpruningexpression(ws_sold_date_sk#61 IN dynamicpruning#62)] +PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct -(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) +(52) CometFilter +Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] +Condition : isnotnull(ws_web_page_sk#50) -(54) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#63] +(53) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#55] -(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 +(54) CometBroadcastHashJoin +Left output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] +Right output [1]: [d_date_sk#55] +Arguments: [ws_sold_date_sk#53], [d_date_sk#55], Inner, BuildRight -(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] +(55) CometProject +Input [5]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53, d_date_sk#55] +Arguments: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52], [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] -(57) Scan parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#64] +(56) Scan parquet spark_catalog.default.web_page +Output [1]: [wp_web_page_sk#56] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct -(58) CometFilter -Input [1]: [wp_web_page_sk#64] -Condition : isnotnull(wp_web_page_sk#64) - -(59) CometBroadcastExchange -Input [1]: [wp_web_page_sk#64] -Arguments: [wp_web_page_sk#64] +(57) CometFilter +Input [1]: [wp_web_page_sk#56] +Condition : isnotnull(wp_web_page_sk#56) -(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 +(58) CometBroadcastExchange +Input [1]: [wp_web_page_sk#56] +Arguments: [wp_web_page_sk#56] -(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] +(59) CometBroadcastHashJoin +Left output [3]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] +Right output [1]: [wp_web_page_sk#56] +Arguments: [ws_web_page_sk#50], [wp_web_page_sk#56], Inner, BuildRight -(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))] +(60) CometProject +Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] +Arguments: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56], [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] -(63) ColumnarToRow [codegen id : 9] -Input [3]: [wp_web_page_sk#64, sum#65, sum#66] +(61) CometHashAggregate +Input [3]: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] +Keys [1]: [wp_web_page_sk#56] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#51)), partial_sum(UnscaledValue(ws_net_profit#52))] -(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] +(62) CometColumnarExchange +Input [3]: [wp_web_page_sk#56, sum#57, sum#58] +Arguments: hashpartitioning(wp_web_page_sk#56, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(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] +(63) CometHashAggregate +Input [3]: [wp_web_page_sk#56, sum#57, sum#58] +Keys [1]: [wp_web_page_sk#56] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#51)), sum(UnscaledValue(ws_net_profit#52))] -(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] +(64) Scan parquet spark_catalog.default.web_returns +Output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#74), dynamicpruningexpression(wr_returned_date_sk#74 IN dynamicpruning#75)] +PartitionFilters: [isnotnull(wr_returned_date_sk#62), dynamicpruningexpression(wr_returned_date_sk#62 IN dynamicpruning#63)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct -(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) +(65) CometFilter +Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] +Condition : isnotnull(wr_web_page_sk#59) -(68) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#76] +(66) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#64] -(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 +(67) CometBroadcastHashJoin +Left output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] +Right output [1]: [d_date_sk#64] +Arguments: [wr_returned_date_sk#62], [d_date_sk#64], Inner, BuildRight -(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] +(68) CometProject +Input [5]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62, d_date_sk#64] +Arguments: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61], [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] -(71) ReusedExchange [Reuses operator id: 59] -Output [1]: [wp_web_page_sk#77] +(69) ReusedExchange [Reuses operator id: 58] +Output [1]: [wp_web_page_sk#65] -(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 +(70) CometBroadcastHashJoin +Left output [3]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] +Right output [1]: [wp_web_page_sk#65] +Arguments: [wr_web_page_sk#59], [wp_web_page_sk#65], 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] +(71) CometProject +Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] +Arguments: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65], [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] -(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 +(72) CometHashAggregate +Input [3]: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] +Keys [1]: [wp_web_page_sk#65] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#60)), partial_sum(UnscaledValue(wr_net_loss#61))] + +(73) CometColumnarExchange +Input [3]: [wp_web_page_sk#65, sum#66, sum#67] +Arguments: hashpartitioning(wp_web_page_sk#65, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(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#128, sum(returns#30)#101 AS returns#129, sum(profit#31)#102 AS profit#130] - -(92) HashAggregate [codegen id : 43] -Input [3]: [sales#128, returns#129, profit#130] +(74) CometHashAggregate +Input [3]: [wp_web_page_sk#65, sum#66, sum#67] +Keys [1]: [wp_web_page_sk#65] +Functions [2]: [sum(UnscaledValue(wr_return_amt#60)), sum(UnscaledValue(wr_net_loss#61))] + +(75) CometBroadcastExchange +Input [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] +Arguments: [wp_web_page_sk#65, returns#68, profit_loss#69] + +(76) CometBroadcastHashJoin +Left output [3]: [wp_web_page_sk#56, sales#70, profit#71] +Right output [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] +Arguments: [wp_web_page_sk#56], [wp_web_page_sk#65], LeftOuter, BuildRight + +(77) CometProject +Input [6]: [wp_web_page_sk#56, sales#70, profit#71, wp_web_page_sk#65, returns#68, profit_loss#69] +Arguments: [channel#72, id#73, sales#70, returns#74, profit#75], [web channel AS channel#72, wp_web_page_sk#56 AS id#73, sales#70, coalesce(returns#68, 0.00) AS returns#74, (profit#71 - coalesce(profit_loss#69, 0.00)) AS profit#75] + +(78) ColumnarToRow [codegen id : 4] +Input [5]: [channel#72, id#73, sales#70, returns#74, profit#75] + +(79) Union + +(80) HashAggregate [codegen id : 5] +Input [5]: [channel#24, id#25, sales#22, returns#26, profit#27] +Keys [2]: [channel#24, id#25] +Functions [3]: [partial_sum(sales#22), partial_sum(returns#26), partial_sum(profit#27)] +Aggregate Attributes [6]: [sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81] +Results [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(81) RowToColumnar +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(82) CometColumnarExchange +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] +Arguments: hashpartitioning(channel#24, id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(83) ColumnarToRow [codegen id : 6] +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(84) HashAggregate [codegen id : 6] +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] +Keys [2]: [channel#24, id#25] +Functions [3]: [sum(sales#22), sum(returns#26), sum(profit#27)] +Aggregate Attributes [3]: [sum(sales#22)#88, sum(returns#26)#89, sum(profit#27)#90] +Results [5]: [channel#24, id#25, cast(sum(sales#22)#88 as decimal(37,2)) AS sales#91, cast(sum(returns#26)#89 as decimal(37,2)) AS returns#92, cast(sum(profit#27)#90 as decimal(38,2)) AS profit#93] + +(85) ReusedExchange [Reuses operator id: 82] +Output [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(86) ColumnarToRow [codegen id : 12] +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(87) HashAggregate [codegen id : 12] +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] +Keys [2]: [channel#24, id#25] +Functions [3]: [sum(sales#22), sum(returns#26), sum(profit#27)] +Aggregate Attributes [3]: [sum(sales#22)#88, sum(returns#26)#89, sum(profit#27)#90] +Results [4]: [channel#24, sum(sales#22)#88 AS sales#94, sum(returns#26)#89 AS returns#95, sum(profit#27)#90 AS profit#96] + +(88) HashAggregate [codegen id : 12] +Input [4]: [channel#24, sales#94, returns#95, profit#96] +Keys [1]: [channel#24] +Functions [3]: [partial_sum(sales#94), partial_sum(returns#95), partial_sum(profit#96)] +Aggregate Attributes [6]: [sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102] +Results [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] + +(89) RowToColumnar +Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] + +(90) CometColumnarExchange +Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] +Arguments: hashpartitioning(channel#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(91) ColumnarToRow [codegen id : 13] +Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] + +(92) HashAggregate [codegen id : 13] +Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] +Keys [1]: [channel#24] +Functions [3]: [sum(sales#94), sum(returns#95), sum(profit#96)] +Aggregate Attributes [3]: [sum(sales#94)#109, sum(returns#95)#110, sum(profit#96)#111] +Results [5]: [channel#24, null AS id#112, sum(sales#94)#109 AS sales#113, sum(returns#95)#110 AS returns#114, sum(profit#96)#111 AS profit#115] + +(93) ReusedExchange [Reuses operator id: 82] +Output [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(94) ColumnarToRow [codegen id : 19] +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(95) HashAggregate [codegen id : 19] +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] +Keys [2]: [channel#24, id#25] +Functions [3]: [sum(sales#22), sum(returns#26), sum(profit#27)] +Aggregate Attributes [3]: [sum(sales#22)#88, sum(returns#26)#89, sum(profit#27)#90] +Results [3]: [sum(sales#22)#88 AS sales#116, sum(returns#26)#89 AS returns#117, sum(profit#27)#90 AS profit#118] + +(96) HashAggregate [codegen id : 19] +Input [3]: [sales#116, returns#117, profit#118] Keys: [] -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 [6]: [sum#137, isEmpty#138, sum#139, isEmpty#140, sum#141, isEmpty#142] +Functions [3]: [partial_sum(sales#116), partial_sum(returns#117), partial_sum(profit#118)] +Aggregate Attributes [6]: [sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] +Results [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] + +(97) RowToColumnar +Input [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] + +(98) CometColumnarExchange +Input [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(93) Exchange -Input [6]: [sum#137, isEmpty#138, sum#139, isEmpty#140, sum#141, isEmpty#142] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] +(99) ColumnarToRow [codegen id : 20] +Input [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] -(94) HashAggregate [codegen id : 44] -Input [6]: [sum#137, isEmpty#138, sum#139, isEmpty#140, sum#141, isEmpty#142] +(100) HashAggregate [codegen id : 20] +Input [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] Keys: [] -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]: [null AS channel#146, null AS id#147, sum(sales#128)#143 AS sales#148, sum(returns#129)#144 AS returns#149, sum(profit#130)#145 AS profit#150] +Functions [3]: [sum(sales#116), sum(returns#117), sum(profit#118)] +Aggregate Attributes [3]: [sum(sales#116)#131, sum(returns#117)#132, sum(profit#118)#133] +Results [5]: [null AS channel#134, null AS id#135, sum(sales#116)#131 AS sales#136, sum(returns#117)#132 AS returns#137, sum(profit#118)#133 AS profit#138] -(95) Union +(101) Union -(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] +(102) HashAggregate [codegen id : 21] +Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] +Keys [5]: [channel#24, id#25, sales#91, returns#92, profit#93] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#28, id#29, sales#103, returns#104, profit#105] +Results [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -(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] +(103) RowToColumnar +Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -(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] +(104) CometColumnarExchange +Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] +Arguments: hashpartitioning(channel#24, id#25, sales#91, returns#92, profit#93, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] + +(105) CometHashAggregate +Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] +Keys [5]: [channel#24, id#25, sales#91, returns#92, profit#93] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#28, id#29, sales#103, returns#104, profit#105] -(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] +(106) CometTakeOrderedAndProject +Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#24 ASC NULLS FIRST,id#25 ASC NULLS FIRST], output=[channel#24,id#25,sales#91,returns#92,profit#93]), [channel#24, id#25, sales#91, returns#92, profit#93], 100, [channel#24 ASC NULLS FIRST, id#25 ASC NULLS FIRST], [channel#24, id#25, sales#91, returns#92, profit#93] + +(107) ColumnarToRow [codegen id : 22] +Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (104) -+- * ColumnarToRow (103) - +- CometProject (102) - +- CometFilter (101) - +- CometScan parquet spark_catalog.default.date_dim (100) +BroadcastExchange (112) ++- * ColumnarToRow (111) + +- CometProject (110) + +- CometFilter (109) + +- CometScan parquet spark_catalog.default.date_dim (108) -(100) Scan parquet spark_catalog.default.date_dim +(108) 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 -(101) CometFilter +(109) 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)) -(102) CometProject +(110) CometProject Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(103) ColumnarToRow [codegen id : 1] +(111) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(104) BroadcastExchange +(112) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -Subquery:2 Hosting operator id = 18 Hosting Expression = sr_returned_date_sk#18 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 17 Hosting Expression = sr_returned_date_sk#14 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#35 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#31 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 42 Hosting Expression = cr_returned_date_sk#46 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 41 Hosting Expression = cr_returned_date_sk#40 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 52 Hosting Expression = ws_sold_date_sk#61 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 51 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 66 Hosting Expression = wr_returned_date_sk#74 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 64 Hosting Expression = wr_returned_date_sk#62 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/simplified.txt index 34972d290..6e5506d68 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/simplified.txt @@ -1,60 +1,56 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (46) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Exchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (45) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - 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 (13) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [s_store_sk,sum,sum,ss_ext_sales_price,ss_net_profit] - CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] - CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_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 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] - InputAdapter - 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] #8 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter +WholeStageCodegen (22) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [channel,id,sales,returns,profit] + CometColumnarExchange [channel,id,sales,returns,profit] #1 + RowToColumnar + WholeStageCodegen (21) + HashAggregate [channel,id,sales,returns,profit] + InputAdapter + Union + WholeStageCodegen (6) + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel,id] #2 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [s_store_sk,returns,profit,profit_loss] [channel,id,sales,returns,profit] + CometBroadcastHashJoin [s_store_sk,sales,profit,s_store_sk,returns,profit_loss] + CometHashAggregate [s_store_sk,sales,profit,sum,sum,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit))] + CometColumnarExchange [s_store_sk] #3 + CometHashAggregate [s_store_sk,sum,sum,ss_ext_sales_price,ss_net_profit] + CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_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 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [s_store_sk] #6 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] + CometBroadcastExchange [s_store_sk,returns,profit_loss] #7 + CometHashAggregate [s_store_sk,returns,profit_loss,sum,sum,sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss))] + CometColumnarExchange [s_store_sk] #8 CometHashAggregate [s_store_sk,sum,sum,sr_return_amt,sr_net_loss] CometProject [sr_return_amt,sr_net_loss,s_store_sk] CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] @@ -65,66 +61,54 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedSubquery [d_date_sk] #1 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 #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] + WholeStageCodegen (3) + Project [cs_call_center_sk,sales,returns,profit,profit_loss] + BroadcastNestedLoopJoin InputAdapter - Exchange [cs_call_center_sk] #10 - WholeStageCodegen (5) + BroadcastExchange #9 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [cs_call_center_sk,sum,sum,cs_ext_sales_price,cs_net_profit] - CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,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 - 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 #11 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,cr_return_amount,cr_net_loss] - CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_return_amount,cr_net_loss,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 - 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] #12 - WholeStageCodegen (9) + CometHashAggregate [cs_call_center_sk,sales,profit,sum,sum,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] + CometColumnarExchange [cs_call_center_sk] #10 + CometHashAggregate [cs_call_center_sk,sum,sum,cs_ext_sales_price,cs_net_profit] + CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,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 + ReusedExchange [d_date_sk] #5 ColumnarToRow InputAdapter - CometHashAggregate [wp_web_page_sk,sum,sum,ws_ext_sales_price,ws_net_profit] - CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk] - CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_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 + CometHashAggregate [returns,profit_loss,sum,sum,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] + CometColumnarExchange #11 + CometHashAggregate [sum,sum,cr_return_amount,cr_net_loss] + CometProject [cr_return_amount,cr_net_loss] + CometBroadcastHashJoin [cr_return_amount,cr_net_loss,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 ReusedExchange [d_date_sk] #5 - CometBroadcastExchange [wp_web_page_sk] #13 - CometFilter [wp_web_page_sk] - CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] - InputAdapter - 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] #15 - WholeStageCodegen (10) - ColumnarToRow - InputAdapter + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometProject [wp_web_page_sk,returns,profit,profit_loss] [channel,id,sales,returns,profit] + CometBroadcastHashJoin [wp_web_page_sk,sales,profit,wp_web_page_sk,returns,profit_loss] + CometHashAggregate [wp_web_page_sk,sales,profit,sum,sum,sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit))] + CometColumnarExchange [wp_web_page_sk] #12 + CometHashAggregate [wp_web_page_sk,sum,sum,ws_ext_sales_price,ws_net_profit] + CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] + CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk] + CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_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 + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange [wp_web_page_sk] #13 + CometFilter [wp_web_page_sk] + CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] + CometBroadcastExchange [wp_web_page_sk,returns,profit_loss] #14 + CometHashAggregate [wp_web_page_sk,returns,profit_loss,sum,sum,sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss))] + CometColumnarExchange [wp_web_page_sk] #15 CometHashAggregate [wp_web_page_sk,sum,sum,wr_return_amt,wr_net_loss] CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] @@ -135,21 +119,27 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedSubquery [d_date_sk] #1 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] #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 (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 #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 - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + WholeStageCodegen (13) + HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel] #16 + RowToColumnar + WholeStageCodegen (12) + 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] + ColumnarToRow + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + WholeStageCodegen (20) + HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange #17 + RowToColumnar + WholeStageCodegen (19) + 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] + ColumnarToRow + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/explain.txt index c7ee5b1c9..9304cb85f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/explain.txt @@ -1,74 +1,71 @@ == Physical Plan == -TakeOrderedAndProject (70) -+- * Project (69) - +- * SortMergeJoin Inner (68) - :- * Project (45) - : +- * SortMergeJoin Inner (44) - : :- * Sort (21) - : : +- * HashAggregate (20) - : : +- Exchange (19) - : : +- * HashAggregate (18) - : : +- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Project (14) - : : : +- * Filter (13) - : : : +- * SortMergeJoin LeftOuter (12) - : : : :- * Sort (5) - : : : : +- Exchange (4) - : : : : +- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- * Sort (11) - : : : +- Exchange (10) - : : : +- * ColumnarToRow (9) - : : : +- CometProject (8) - : : : +- CometFilter (7) - : : : +- CometScan parquet spark_catalog.default.store_returns (6) - : : +- ReusedExchange (15) - : +- * Sort (43) - : +- * Filter (42) - : +- * HashAggregate (41) - : +- Exchange (40) - : +- * HashAggregate (39) - : +- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * Project (35) - : : +- * Filter (34) - : : +- * SortMergeJoin LeftOuter (33) - : : :- * Sort (26) - : : : +- Exchange (25) - : : : +- * ColumnarToRow (24) - : : : +- CometFilter (23) - : : : +- CometScan parquet spark_catalog.default.web_sales (22) - : : +- * Sort (32) - : : +- Exchange (31) - : : +- * ColumnarToRow (30) - : : +- CometProject (29) - : : +- CometFilter (28) - : : +- CometScan parquet spark_catalog.default.web_returns (27) - : +- ReusedExchange (36) - +- * Sort (67) - +- * Filter (66) - +- * HashAggregate (65) - +- Exchange (64) - +- * HashAggregate (63) - +- * Project (62) - +- * BroadcastHashJoin Inner BuildRight (61) - :- * Project (59) - : +- * Filter (58) - : +- * SortMergeJoin LeftOuter (57) - : :- * Sort (50) - : : +- Exchange (49) - : : +- * ColumnarToRow (48) - : : +- CometFilter (47) - : : +- CometScan parquet spark_catalog.default.catalog_sales (46) - : +- * Sort (56) - : +- Exchange (55) - : +- * ColumnarToRow (54) - : +- CometProject (53) - : +- CometFilter (52) - : +- CometScan parquet spark_catalog.default.catalog_returns (51) - +- ReusedExchange (60) +TakeOrderedAndProject (67) ++- * Project (66) + +- * ColumnarToRow (65) + +- CometSortMergeJoin (64) + :- CometProject (43) + : +- CometSortMergeJoin (42) + : :- CometSort (21) + : : +- CometHashAggregate (20) + : : +- CometColumnarExchange (19) + : : +- CometHashAggregate (18) + : : +- CometProject (17) + : : +- CometBroadcastHashJoin (16) + : : :- CometProject (12) + : : : +- CometFilter (11) + : : : +- CometSortMergeJoin (10) + : : : :- CometSort (4) + : : : : +- CometColumnarExchange (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometSort (9) + : : : +- CometColumnarExchange (8) + : : : +- CometProject (7) + : : : +- CometFilter (6) + : : : +- CometScan parquet spark_catalog.default.store_returns (5) + : : +- CometBroadcastExchange (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.date_dim (13) + : +- CometSort (41) + : +- CometFilter (40) + : +- CometHashAggregate (39) + : +- CometColumnarExchange (38) + : +- CometHashAggregate (37) + : +- CometProject (36) + : +- CometBroadcastHashJoin (35) + : :- CometProject (33) + : : +- CometFilter (32) + : : +- CometSortMergeJoin (31) + : : :- CometSort (25) + : : : +- CometColumnarExchange (24) + : : : +- CometFilter (23) + : : : +- CometScan parquet spark_catalog.default.web_sales (22) + : : +- CometSort (30) + : : +- CometColumnarExchange (29) + : : +- CometProject (28) + : : +- CometFilter (27) + : : +- CometScan parquet spark_catalog.default.web_returns (26) + : +- ReusedExchange (34) + +- CometSort (63) + +- CometFilter (62) + +- CometHashAggregate (61) + +- CometColumnarExchange (60) + +- CometHashAggregate (59) + +- CometProject (58) + +- CometBroadcastHashJoin (57) + :- CometProject (55) + : +- CometFilter (54) + : +- CometSortMergeJoin (53) + : :- CometSort (47) + : : +- CometColumnarExchange (46) + : : +- CometFilter (45) + : : +- CometScan parquet spark_catalog.default.catalog_sales (44) + : +- CometSort (52) + : +- CometColumnarExchange (51) + : +- CometProject (50) + : +- CometFilter (49) + : +- CometScan parquet spark_catalog.default.catalog_returns (48) + +- ReusedExchange (56) (1) Scan parquet spark_catalog.default.store_sales @@ -83,349 +80,326 @@ ReadSchema: struct -(7) CometFilter +(6) CometFilter Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] Condition : (isnotnull(sr_ticket_number#10) AND isnotnull(sr_item_sk#9)) -(8) CometProject +(7) CometProject Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_item_sk#9, sr_ticket_number#10] -(9) ColumnarToRow [codegen id : 3] -Input [2]: [sr_item_sk#9, sr_ticket_number#10] - -(10) Exchange +(8) CometColumnarExchange Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(11) Sort [codegen id : 4] +(9) CometSort Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST], false, 0 +Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST] -(12) SortMergeJoin [codegen id : 6] -Left keys [2]: [ss_ticket_number#3, ss_item_sk#1] -Right keys [2]: [sr_ticket_number#10, sr_item_sk#9] -Join type: LeftOuter -Join condition: None +(10) CometSortMergeJoin +Left output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] +Right output [2]: [sr_item_sk#9, sr_ticket_number#10] +Arguments: [ss_ticket_number#3, ss_item_sk#1], [sr_ticket_number#10, sr_item_sk#9], LeftOuter -(13) Filter [codegen id : 6] +(11) CometFilter Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] Condition : isnull(sr_ticket_number#10) -(14) Project [codegen id : 6] -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] +(12) CometProject Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -(15) ReusedExchange [Reuses operator id: 74] +(13) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#12, d_year#13] +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 + +(14) CometFilter +Input [2]: [d_date_sk#12, d_year#13] +Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) + +(15) CometBroadcastExchange +Input [2]: [d_date_sk#12, d_year#13] +Arguments: [d_date_sk#12, d_year#13] -(16) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] +Right output [2]: [d_date_sk#12, d_year#13] +Arguments: [ss_sold_date_sk#7], [d_date_sk#12], Inner, BuildRight -(17) Project [codegen id : 6] -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] +(17) CometProject Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] -(18) HashAggregate [codegen id : 6] +(18) CometHashAggregate Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesale_cost#5)), partial_sum(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [3]: [sum#14, sum#15, sum#16] -Results [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] -(19) Exchange -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] -Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(19) CometColumnarExchange +Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] +Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(20) HashAggregate [codegen id : 7] -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] +(20) CometHashAggregate +Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [3]: [sum(ss_quantity#4)#20, sum(UnscaledValue(ss_wholesale_cost#5))#21, sum(UnscaledValue(ss_sales_price#6))#22] -Results [6]: [d_year#13 AS ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, sum(ss_quantity#4)#20 AS ss_qty#24, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#5))#21,17,2) AS ss_wc#25, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#22,17,2) AS ss_sp#26] -(21) Sort [codegen id : 7] -Input [6]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] -Arguments: [ss_sold_year#23 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], false, 0 +(21) CometSort +Input [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] +Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20], [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST] (22) Scan parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] 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#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#28)] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (23) CometFilter -Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Condition : (isnotnull(ws_item_sk#27) AND isnotnull(ws_bill_customer_sk#28)) +Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] +Condition : (isnotnull(ws_item_sk#21) AND isnotnull(ws_bill_customer_sk#22)) -(24) ColumnarToRow [codegen id : 8] -Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +(24) CometColumnarExchange +Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] +Arguments: hashpartitioning(ws_order_number#23, ws_item_sk#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(25) Exchange -Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Arguments: hashpartitioning(ws_order_number#29, ws_item_sk#27, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(25) CometSort +Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] +Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_order_number#23 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST] -(26) Sort [codegen id : 9] -Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Arguments: [ws_order_number#29 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST], false, 0 - -(27) Scan parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] +(26) Scan parquet spark_catalog.default.web_returns +Output [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct -(28) CometFilter -Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] -Condition : (isnotnull(wr_order_number#36) AND isnotnull(wr_item_sk#35)) - -(29) CometProject -Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] -Arguments: [wr_item_sk#35, wr_order_number#36], [wr_item_sk#35, wr_order_number#36] - -(30) ColumnarToRow [codegen id : 10] -Input [2]: [wr_item_sk#35, wr_order_number#36] - -(31) Exchange -Input [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: hashpartitioning(wr_order_number#36, wr_item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(32) Sort [codegen id : 11] -Input [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: [wr_order_number#36 ASC NULLS FIRST, wr_item_sk#35 ASC NULLS FIRST], false, 0 - -(33) SortMergeJoin [codegen id : 13] -Left keys [2]: [ws_order_number#29, ws_item_sk#27] -Right keys [2]: [wr_order_number#36, wr_item_sk#35] -Join type: LeftOuter -Join condition: None - -(34) Filter [codegen id : 13] -Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] -Condition : isnull(wr_order_number#36) - -(35) Project [codegen id : 13] -Output [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] - -(36) ReusedExchange [Reuses operator id: 74] -Output [2]: [d_date_sk#38, d_year#39] - -(37) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#38] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 13] -Output [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] -Input [8]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, d_date_sk#38, d_year#39] - -(39) HashAggregate [codegen id : 13] -Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] -Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] -Functions [3]: [partial_sum(ws_quantity#30), partial_sum(UnscaledValue(ws_wholesale_cost#31)), partial_sum(UnscaledValue(ws_sales_price#32))] -Aggregate Attributes [3]: [sum#40, sum#41, sum#42] -Results [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] - -(40) Exchange -Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] -Arguments: hashpartitioning(d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, 5), ENSURE_REQUIREMENTS, [plan_id=6] - -(41) HashAggregate [codegen id : 14] -Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] -Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] -Functions [3]: [sum(ws_quantity#30), sum(UnscaledValue(ws_wholesale_cost#31)), sum(UnscaledValue(ws_sales_price#32))] -Aggregate Attributes [3]: [sum(ws_quantity#30)#46, sum(UnscaledValue(ws_wholesale_cost#31))#47, sum(UnscaledValue(ws_sales_price#32))#48] -Results [6]: [d_year#39 AS ws_sold_year#49, ws_item_sk#27, ws_bill_customer_sk#28 AS ws_customer_sk#50, sum(ws_quantity#30)#46 AS ws_qty#51, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#31))#47,17,2) AS ws_wc#52, MakeDecimal(sum(UnscaledValue(ws_sales_price#32))#48,17,2) AS ws_sp#53] - -(42) Filter [codegen id : 14] -Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] -Condition : (coalesce(ws_qty#51, 0) > 0) - -(43) Sort [codegen id : 14] -Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] -Arguments: [ws_sold_year#49 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST, ws_customer_sk#50 ASC NULLS FIRST], false, 0 - -(44) SortMergeJoin [codegen id : 15] -Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50] -Join type: Inner -Join condition: None - -(45) Project [codegen id : 15] -Output [9]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53] -Input [12]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] - -(46) Scan parquet spark_catalog.default.catalog_sales -Output [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +(27) CometFilter +Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] +Condition : (isnotnull(wr_order_number#30) AND isnotnull(wr_item_sk#29)) + +(28) CometProject +Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] +Arguments: [wr_item_sk#29, wr_order_number#30], [wr_item_sk#29, wr_order_number#30] + +(29) CometColumnarExchange +Input [2]: [wr_item_sk#29, wr_order_number#30] +Arguments: hashpartitioning(wr_order_number#30, wr_item_sk#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(30) CometSort +Input [2]: [wr_item_sk#29, wr_order_number#30] +Arguments: [wr_item_sk#29, wr_order_number#30], [wr_order_number#30 ASC NULLS FIRST, wr_item_sk#29 ASC NULLS FIRST] + +(31) CometSortMergeJoin +Left output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] +Right output [2]: [wr_item_sk#29, wr_order_number#30] +Arguments: [ws_order_number#23, ws_item_sk#21], [wr_order_number#30, wr_item_sk#29], LeftOuter + +(32) CometFilter +Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] +Condition : isnull(wr_order_number#30) + +(33) CometProject +Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] +Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] + +(34) ReusedExchange [Reuses operator id: 15] +Output [2]: [d_date_sk#32, d_year#33] + +(35) CometBroadcastHashJoin +Left output [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] +Right output [2]: [d_date_sk#32, d_year#33] +Arguments: [ws_sold_date_sk#27], [d_date_sk#32], Inner, BuildRight + +(36) CometProject +Input [8]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, d_date_sk#32, d_year#33] +Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] + +(37) CometHashAggregate +Input [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] +Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] +Functions [3]: [partial_sum(ws_quantity#24), partial_sum(UnscaledValue(ws_wholesale_cost#25)), partial_sum(UnscaledValue(ws_sales_price#26))] + +(38) CometColumnarExchange +Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] +Arguments: hashpartitioning(d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(39) CometHashAggregate +Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] +Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] +Functions [3]: [sum(ws_quantity#24), sum(UnscaledValue(ws_wholesale_cost#25)), sum(UnscaledValue(ws_sales_price#26))] + +(40) CometFilter +Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] +Condition : (coalesce(ws_qty#39, 0) > 0) + +(41) CometSort +Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] +Arguments: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41], [ws_sold_year#37 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST, ws_customer_sk#38 ASC NULLS FIRST] + +(42) CometSortMergeJoin +Left output [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] +Right output [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] +Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38], Inner + +(43) CometProject +Input [12]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] +Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41], [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] + +(44) Scan parquet spark_catalog.default.catalog_sales +Output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_sold_date_sk#60 IN dynamicpruning#61)] +PartitionFilters: [isnotnull(cs_sold_date_sk#48), dynamicpruningexpression(cs_sold_date_sk#48 IN dynamicpruning#49)] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(47) CometFilter -Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Condition : (isnotnull(cs_item_sk#55) AND isnotnull(cs_bill_customer_sk#54)) - -(48) ColumnarToRow [codegen id : 16] -Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +(45) CometFilter +Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] +Condition : (isnotnull(cs_item_sk#43) AND isnotnull(cs_bill_customer_sk#42)) -(49) Exchange -Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Arguments: hashpartitioning(cs_order_number#56, cs_item_sk#55, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(46) CometColumnarExchange +Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] +Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(50) Sort [codegen id : 17] -Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Arguments: [cs_order_number#56 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST], false, 0 +(47) CometSort +Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] +Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_order_number#44 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST] -(51) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] +(48) Scan parquet spark_catalog.default.catalog_returns +Output [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(52) CometFilter -Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] -Condition : (isnotnull(cr_order_number#63) AND isnotnull(cr_item_sk#62)) - -(53) CometProject -Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] -Arguments: [cr_item_sk#62, cr_order_number#63], [cr_item_sk#62, cr_order_number#63] - -(54) ColumnarToRow [codegen id : 18] -Input [2]: [cr_item_sk#62, cr_order_number#63] - -(55) Exchange -Input [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: hashpartitioning(cr_order_number#63, cr_item_sk#62, 5), ENSURE_REQUIREMENTS, [plan_id=8] - -(56) Sort [codegen id : 19] -Input [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: [cr_order_number#63 ASC NULLS FIRST, cr_item_sk#62 ASC NULLS FIRST], false, 0 - -(57) SortMergeJoin [codegen id : 21] -Left keys [2]: [cs_order_number#56, cs_item_sk#55] -Right keys [2]: [cr_order_number#63, cr_item_sk#62] -Join type: LeftOuter -Join condition: None - -(58) Filter [codegen id : 21] -Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] -Condition : isnull(cr_order_number#63) - -(59) Project [codegen id : 21] -Output [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] - -(60) ReusedExchange [Reuses operator id: 74] -Output [2]: [d_date_sk#65, d_year#66] - -(61) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [cs_sold_date_sk#60] -Right keys [1]: [d_date_sk#65] -Join type: Inner -Join condition: None - -(62) Project [codegen id : 21] -Output [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] -Input [8]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, d_date_sk#65, d_year#66] - -(63) HashAggregate [codegen id : 21] -Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] -Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] -Functions [3]: [partial_sum(cs_quantity#57), partial_sum(UnscaledValue(cs_wholesale_cost#58)), partial_sum(UnscaledValue(cs_sales_price#59))] -Aggregate Attributes [3]: [sum#67, sum#68, sum#69] -Results [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] - -(64) Exchange -Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] -Arguments: hashpartitioning(d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, 5), ENSURE_REQUIREMENTS, [plan_id=9] - -(65) HashAggregate [codegen id : 22] -Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] -Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] -Functions [3]: [sum(cs_quantity#57), sum(UnscaledValue(cs_wholesale_cost#58)), sum(UnscaledValue(cs_sales_price#59))] -Aggregate Attributes [3]: [sum(cs_quantity#57)#73, sum(UnscaledValue(cs_wholesale_cost#58))#74, sum(UnscaledValue(cs_sales_price#59))#75] -Results [6]: [d_year#66 AS cs_sold_year#76, cs_item_sk#55, cs_bill_customer_sk#54 AS cs_customer_sk#77, sum(cs_quantity#57)#73 AS cs_qty#78, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#58))#74,17,2) AS cs_wc#79, MakeDecimal(sum(UnscaledValue(cs_sales_price#59))#75,17,2) AS cs_sp#80] - -(66) Filter [codegen id : 22] -Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] -Condition : (coalesce(cs_qty#78, 0) > 0) - -(67) Sort [codegen id : 22] -Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] -Arguments: [cs_sold_year#76 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST, cs_customer_sk#77 ASC NULLS FIRST], false, 0 - -(68) SortMergeJoin [codegen id : 23] -Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77] -Join type: Inner -Join condition: None - -(69) Project [codegen id : 23] -Output [13]: [round((cast(ss_qty#24 as double) / cast(coalesce((ws_qty#51 + cs_qty#78), 1) as double)), 2) AS ratio#81, ss_qty#24 AS store_qty#82, ss_wc#25 AS store_wholesale_cost#83, ss_sp#26 AS store_sales_price#84, (coalesce(ws_qty#51, 0) + coalesce(cs_qty#78, 0)) AS other_chan_qty#85, (coalesce(ws_wc#52, 0.00) + coalesce(cs_wc#79, 0.00)) AS other_chan_wholesale_cost#86, (coalesce(ws_sp#53, 0.00) + coalesce(cs_sp#80, 0.00)) AS other_chan_sales_price#87, ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] -Input [15]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53, cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] - -(70) TakeOrderedAndProject -Input [13]: [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87, ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] -Arguments: 100, [ss_sold_year#23 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#24 DESC NULLS LAST, ss_wc#25 DESC NULLS LAST, ss_sp#26 DESC NULLS LAST, other_chan_qty#85 ASC NULLS FIRST, other_chan_wholesale_cost#86 ASC NULLS FIRST, other_chan_sales_price#87 ASC NULLS FIRST, ratio#81 ASC NULLS FIRST], [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87] +(49) CometFilter +Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] +Condition : (isnotnull(cr_order_number#51) AND isnotnull(cr_item_sk#50)) + +(50) CometProject +Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] +Arguments: [cr_item_sk#50, cr_order_number#51], [cr_item_sk#50, cr_order_number#51] + +(51) CometColumnarExchange +Input [2]: [cr_item_sk#50, cr_order_number#51] +Arguments: hashpartitioning(cr_order_number#51, cr_item_sk#50, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(52) CometSort +Input [2]: [cr_item_sk#50, cr_order_number#51] +Arguments: [cr_item_sk#50, cr_order_number#51], [cr_order_number#51 ASC NULLS FIRST, cr_item_sk#50 ASC NULLS FIRST] + +(53) CometSortMergeJoin +Left output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] +Right output [2]: [cr_item_sk#50, cr_order_number#51] +Arguments: [cs_order_number#44, cs_item_sk#43], [cr_order_number#51, cr_item_sk#50], LeftOuter + +(54) CometFilter +Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] +Condition : isnull(cr_order_number#51) + +(55) CometProject +Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] +Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] + +(56) ReusedExchange [Reuses operator id: 15] +Output [2]: [d_date_sk#53, d_year#54] + +(57) CometBroadcastHashJoin +Left output [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] +Right output [2]: [d_date_sk#53, d_year#54] +Arguments: [cs_sold_date_sk#48], [d_date_sk#53], Inner, BuildRight + +(58) CometProject +Input [8]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, d_date_sk#53, d_year#54] +Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] + +(59) CometHashAggregate +Input [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] +Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] +Functions [3]: [partial_sum(cs_quantity#45), partial_sum(UnscaledValue(cs_wholesale_cost#46)), partial_sum(UnscaledValue(cs_sales_price#47))] + +(60) CometColumnarExchange +Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] +Arguments: hashpartitioning(d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(61) CometHashAggregate +Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] +Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] +Functions [3]: [sum(cs_quantity#45), sum(UnscaledValue(cs_wholesale_cost#46)), sum(UnscaledValue(cs_sales_price#47))] + +(62) CometFilter +Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] +Condition : (coalesce(cs_qty#60, 0) > 0) + +(63) CometSort +Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] +Arguments: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62], [cs_sold_year#58 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST, cs_customer_sk#59 ASC NULLS FIRST] + +(64) CometSortMergeJoin +Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] +Right output [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] +Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59], Inner + +(65) ColumnarToRow [codegen id : 1] +Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] + +(66) Project [codegen id : 1] +Output [13]: [round((cast(ss_qty#18 as double) / cast(coalesce((ws_qty#39 + cs_qty#60), 1) as double)), 2) AS ratio#63, ss_qty#18 AS store_qty#64, ss_wc#19 AS store_wholesale_cost#65, ss_sp#20 AS store_sales_price#66, (coalesce(ws_qty#39, 0) + coalesce(cs_qty#60, 0)) AS other_chan_qty#67, (coalesce(ws_wc#40, 0.00) + coalesce(cs_wc#61, 0.00)) AS other_chan_wholesale_cost#68, (coalesce(ws_sp#41, 0.00) + coalesce(cs_sp#62, 0.00)) AS other_chan_sales_price#69, ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] +Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] + +(67) TakeOrderedAndProject +Input [13]: [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69, ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] +Arguments: 100, [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 DESC NULLS LAST, ss_sp#20 DESC NULLS LAST, other_chan_qty#67 ASC NULLS FIRST, other_chan_wholesale_cost#68 ASC NULLS FIRST, other_chan_sales_price#69 ASC NULLS FIRST, ratio#63 ASC NULLS FIRST], [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (74) -+- * ColumnarToRow (73) - +- CometFilter (72) - +- CometScan parquet spark_catalog.default.date_dim (71) +BroadcastExchange (71) ++- * ColumnarToRow (70) + +- CometFilter (69) + +- CometScan parquet spark_catalog.default.date_dim (68) -(71) Scan parquet spark_catalog.default.date_dim +(68) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#12, d_year#13] 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 -(72) CometFilter +(69) CometFilter Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(73) ColumnarToRow [codegen id : 1] +(70) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] -(74) BroadcastExchange +(71) BroadcastExchange Input [2]: [d_date_sk#12, d_year#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#48 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/simplified.txt index b8e6e0605..6f7cc2edf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/simplified.txt @@ -1,127 +1,78 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ratio,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (23) + WholeStageCodegen (1) Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp,ss_sold_year,ss_item_sk,ss_customer_sk] - SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,cs_sold_year,cs_item_sk,cs_customer_sk] + ColumnarToRow InputAdapter - WholeStageCodegen (15) - Project [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] - SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ws_sold_year,ws_item_sk,ws_customer_sk] - InputAdapter - WholeStageCodegen (7) - Sort [ss_sold_year,ss_item_sk,ss_customer_sk] - HashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] [sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price)),ss_sold_year,ss_qty,ss_wc,ss_sp,sum,sum,sum] - InputAdapter - Exchange [d_year,ss_item_sk,ss_customer_sk] #1 - WholeStageCodegen (6) - HashAggregate [d_year,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] [sum,sum,sum,sum,sum,sum] - Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - Filter [sr_ticket_number] - SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - WholeStageCodegen (2) - Sort [ss_ticket_number,ss_item_sk] - InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - WholeStageCodegen (4) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - Exchange [sr_ticket_number,sr_item_sk] #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - WholeStageCodegen (14) - Sort [ws_sold_year,ws_item_sk,ws_customer_sk] - Filter [ws_qty] - HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] [sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price)),ws_sold_year,ws_customer_sk,ws_qty,ws_wc,ws_sp,sum,sum,sum] - InputAdapter - Exchange [d_year,ws_item_sk,ws_bill_customer_sk] #5 - WholeStageCodegen (13) - HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] [sum,sum,sum,sum,sum,sum] - Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - Filter [wr_order_number] - SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - InputAdapter - WholeStageCodegen (9) - Sort [ws_order_number,ws_item_sk] - InputAdapter - Exchange [ws_order_number,ws_item_sk] #6 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - WholeStageCodegen (11) - Sort [wr_order_number,wr_item_sk] - InputAdapter - Exchange [wr_order_number,wr_item_sk] #7 - WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - WholeStageCodegen (22) - Sort [cs_sold_year,cs_item_sk,cs_customer_sk] - Filter [cs_qty] - HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] [sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price)),cs_sold_year,cs_customer_sk,cs_qty,cs_wc,cs_sp,sum,sum,sum] - InputAdapter - Exchange [d_year,cs_item_sk,cs_bill_customer_sk] #8 - WholeStageCodegen (21) - HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] [sum,sum,sum,sum,sum,sum] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - Filter [cr_order_number] - SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - InputAdapter - WholeStageCodegen (17) - Sort [cs_order_number,cs_item_sk] - InputAdapter - Exchange [cs_order_number,cs_item_sk] #9 - WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - WholeStageCodegen (19) - Sort [cr_order_number,cr_item_sk] - InputAdapter - Exchange [cr_order_number,cr_item_sk] #10 - WholeStageCodegen (18) + CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] + CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] + CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] + CometSort [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp] + CometHashAggregate [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,d_year,sum,sum,sum,sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price))] + CometColumnarExchange [d_year,ss_item_sk,ss_customer_sk] #1 + CometHashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum,ss_quantity,ss_wholesale_cost,ss_sales_price] + CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #2 + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometSort [sr_item_sk,sr_ticket_number] + CometColumnarExchange [sr_ticket_number,sr_item_sk] #4 + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #5 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometSort [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] + CometFilter [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] + CometHashAggregate [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp,d_year,ws_bill_customer_sk,sum,sum,sum,sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price))] + CometColumnarExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 + CometHashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum,ws_quantity,ws_wholesale_cost,ws_sales_price] + CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] + CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,d_date_sk,d_year] + CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] + CometSortMergeJoin [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] + CometSort [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometColumnarExchange [ws_order_number,ws_item_sk] #7 + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_item_sk,wr_order_number] + CometColumnarExchange [wr_order_number,wr_item_sk] #8 + CometProject [wr_item_sk,wr_order_number] + CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] + ReusedExchange [d_date_sk,d_year] #5 + CometSort [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] + CometFilter [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] + CometHashAggregate [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp,d_year,cs_bill_customer_sk,sum,sum,sum,sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price))] + CometColumnarExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 + CometHashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum,cs_quantity,cs_wholesale_cost,cs_sales_price] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] + CometSortMergeJoin [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] + CometSort [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + CometColumnarExchange [cs_order_number,cs_item_sk] #10 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [cr_item_sk,cr_order_number] + CometColumnarExchange [cr_order_number,cr_item_sk] #11 + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] + ReusedExchange [d_date_sk,d_year] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/explain.txt index afa04b0c1..698095377 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/explain.txt @@ -1,124 +1,129 @@ == Physical Plan == -TakeOrderedAndProject (120) -+- * HashAggregate (119) - +- Exchange (118) - +- * HashAggregate (117) - +- Union (116) - :- * HashAggregate (105) - : +- Exchange (104) - : +- * HashAggregate (103) - : +- Union (102) - : :- * HashAggregate (39) - : : +- Exchange (38) - : : +- * HashAggregate (37) - : : +- * Project (36) - : : +- * BroadcastHashJoin Inner BuildRight (35) - : : :- * Project (29) - : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : :- * Project (22) - : : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : : :- * Project (16) - : : : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : : : :- * Project (13) - : : : : : : +- * SortMergeJoin LeftOuter (12) - : : : : : : :- * Sort (5) - : : : : : : : +- Exchange (4) - : : : : : : : +- * ColumnarToRow (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : +- * Sort (11) - : : : : : : +- Exchange (10) - : : : : : : +- * ColumnarToRow (9) - : : : : : : +- CometProject (8) - : : : : : : +- CometFilter (7) - : : : : : : +- CometScan parquet spark_catalog.default.store_returns (6) - : : : : : +- ReusedExchange (14) - : : : : +- BroadcastExchange (20) - : : : : +- * ColumnarToRow (19) - : : : : +- CometFilter (18) - : : : : +- CometScan parquet spark_catalog.default.store (17) - : : : +- BroadcastExchange (27) - : : : +- * ColumnarToRow (26) - : : : +- CometProject (25) - : : : +- CometFilter (24) - : : : +- CometScan parquet spark_catalog.default.item (23) - : : +- BroadcastExchange (34) - : : +- * ColumnarToRow (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan parquet spark_catalog.default.promotion (30) - : :- * HashAggregate (70) - : : +- Exchange (69) - : : +- * HashAggregate (68) - : : +- * Project (67) - : : +- * BroadcastHashJoin Inner BuildRight (66) - : : :- * Project (64) - : : : +- * BroadcastHashJoin Inner BuildRight (63) - : : : :- * Project (61) - : : : : +- * BroadcastHashJoin Inner BuildRight (60) - : : : : :- * Project (55) - : : : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : : : :- * Project (52) - : : : : : : +- * SortMergeJoin LeftOuter (51) - : : : : : : :- * Sort (44) - : : : : : : : +- Exchange (43) - : : : : : : : +- * ColumnarToRow (42) - : : : : : : : +- CometFilter (41) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (40) - : : : : : : +- * Sort (50) - : : : : : : +- Exchange (49) - : : : : : : +- * ColumnarToRow (48) - : : : : : : +- CometProject (47) - : : : : : : +- CometFilter (46) - : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (45) - : : : : : +- ReusedExchange (53) - : : : : +- BroadcastExchange (59) - : : : : +- * ColumnarToRow (58) - : : : : +- CometFilter (57) - : : : : +- CometScan parquet spark_catalog.default.catalog_page (56) - : : : +- ReusedExchange (62) - : : +- ReusedExchange (65) - : +- * HashAggregate (101) - : +- Exchange (100) - : +- * HashAggregate (99) - : +- * Project (98) - : +- * BroadcastHashJoin Inner BuildRight (97) - : :- * Project (95) - : : +- * BroadcastHashJoin Inner BuildRight (94) - : : :- * Project (92) - : : : +- * BroadcastHashJoin Inner BuildRight (91) - : : : :- * Project (86) - : : : : +- * BroadcastHashJoin Inner BuildRight (85) - : : : : :- * Project (83) - : : : : : +- * SortMergeJoin LeftOuter (82) - : : : : : :- * Sort (75) - : : : : : : +- Exchange (74) - : : : : : : +- * ColumnarToRow (73) - : : : : : : +- CometFilter (72) - : : : : : : +- CometScan parquet spark_catalog.default.web_sales (71) - : : : : : +- * Sort (81) - : : : : : +- Exchange (80) - : : : : : +- * ColumnarToRow (79) - : : : : : +- CometProject (78) - : : : : : +- CometFilter (77) - : : : : : +- CometScan parquet spark_catalog.default.web_returns (76) - : : : : +- ReusedExchange (84) - : : : +- BroadcastExchange (90) - : : : +- * ColumnarToRow (89) - : : : +- CometFilter (88) - : : : +- CometScan parquet spark_catalog.default.web_site (87) - : : +- ReusedExchange (93) - : +- ReusedExchange (96) - :- * HashAggregate (110) - : +- Exchange (109) - : +- * HashAggregate (108) - : +- * HashAggregate (107) - : +- ReusedExchange (106) - +- * HashAggregate (115) - +- Exchange (114) - +- * HashAggregate (113) - +- * HashAggregate (112) - +- ReusedExchange (111) +* ColumnarToRow (125) ++- CometTakeOrderedAndProject (124) + +- CometHashAggregate (123) + +- CometColumnarExchange (122) + +- RowToColumnar (121) + +- * HashAggregate (120) + +- Union (119) + :- * HashAggregate (102) + : +- * ColumnarToRow (101) + : +- CometColumnarExchange (100) + : +- RowToColumnar (99) + : +- * HashAggregate (98) + : +- Union (97) + : :- * HashAggregate (38) + : : +- * ColumnarToRow (37) + : : +- CometColumnarExchange (36) + : : +- CometHashAggregate (35) + : : +- CometProject (34) + : : +- CometBroadcastHashJoin (33) + : : :- CometProject (28) + : : : +- CometBroadcastHashJoin (27) + : : : :- CometProject (22) + : : : : +- CometBroadcastHashJoin (21) + : : : : :- CometProject (17) + : : : : : +- CometBroadcastHashJoin (16) + : : : : : :- CometProject (11) + : : : : : : +- CometSortMergeJoin (10) + : : : : : : :- CometSort (4) + : : : : : : : +- CometColumnarExchange (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- CometSort (9) + : : : : : : +- CometColumnarExchange (8) + : : : : : : +- CometProject (7) + : : : : : : +- CometFilter (6) + : : : : : : +- CometScan parquet spark_catalog.default.store_returns (5) + : : : : : +- CometBroadcastExchange (15) + : : : : : +- CometProject (14) + : : : : : +- CometFilter (13) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (12) + : : : : +- CometBroadcastExchange (20) + : : : : +- CometFilter (19) + : : : : +- CometScan parquet spark_catalog.default.store (18) + : : : +- CometBroadcastExchange (26) + : : : +- CometProject (25) + : : : +- CometFilter (24) + : : : +- CometScan parquet spark_catalog.default.item (23) + : : +- CometBroadcastExchange (32) + : : +- CometProject (31) + : : +- CometFilter (30) + : : +- CometScan parquet spark_catalog.default.promotion (29) + : :- * HashAggregate (67) + : : +- * ColumnarToRow (66) + : : +- CometColumnarExchange (65) + : : +- CometHashAggregate (64) + : : +- CometProject (63) + : : +- CometBroadcastHashJoin (62) + : : :- CometProject (60) + : : : +- CometBroadcastHashJoin (59) + : : : :- CometProject (57) + : : : : +- CometBroadcastHashJoin (56) + : : : : :- CometProject (52) + : : : : : +- CometBroadcastHashJoin (51) + : : : : : :- CometProject (49) + : : : : : : +- CometSortMergeJoin (48) + : : : : : : :- CometSort (42) + : : : : : : : +- CometColumnarExchange (41) + : : : : : : : +- CometFilter (40) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (39) + : : : : : : +- CometSort (47) + : : : : : : +- CometColumnarExchange (46) + : : : : : : +- CometProject (45) + : : : : : : +- CometFilter (44) + : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (43) + : : : : : +- ReusedExchange (50) + : : : : +- CometBroadcastExchange (55) + : : : : +- CometFilter (54) + : : : : +- CometScan parquet spark_catalog.default.catalog_page (53) + : : : +- ReusedExchange (58) + : : +- ReusedExchange (61) + : +- * HashAggregate (96) + : +- * ColumnarToRow (95) + : +- CometColumnarExchange (94) + : +- CometHashAggregate (93) + : +- CometProject (92) + : +- CometBroadcastHashJoin (91) + : :- CometProject (89) + : : +- CometBroadcastHashJoin (88) + : : :- CometProject (86) + : : : +- CometBroadcastHashJoin (85) + : : : :- CometProject (81) + : : : : +- CometBroadcastHashJoin (80) + : : : : :- CometProject (78) + : : : : : +- CometSortMergeJoin (77) + : : : : : :- CometSort (71) + : : : : : : +- CometColumnarExchange (70) + : : : : : : +- CometFilter (69) + : : : : : : +- CometScan parquet spark_catalog.default.web_sales (68) + : : : : : +- CometSort (76) + : : : : : +- CometColumnarExchange (75) + : : : : : +- CometProject (74) + : : : : : +- CometFilter (73) + : : : : : +- CometScan parquet spark_catalog.default.web_returns (72) + : : : : +- ReusedExchange (79) + : : : +- CometBroadcastExchange (84) + : : : +- CometFilter (83) + : : : +- CometScan parquet spark_catalog.default.web_site (82) + : : +- ReusedExchange (87) + : +- ReusedExchange (90) + :- * HashAggregate (110) + : +- * ColumnarToRow (109) + : +- CometColumnarExchange (108) + : +- RowToColumnar (107) + : +- * HashAggregate (106) + : +- * HashAggregate (105) + : +- * ColumnarToRow (104) + : +- ReusedExchange (103) + +- * HashAggregate (118) + +- * ColumnarToRow (117) + +- CometColumnarExchange (116) + +- RowToColumnar (115) + +- * HashAggregate (114) + +- * HashAggregate (113) + +- * ColumnarToRow (112) + +- ReusedExchange (111) (1) Scan parquet spark_catalog.default.store_sales @@ -133,598 +138,597 @@ ReadSchema: struct -(7) CometFilter +(6) CometFilter Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] Condition : (isnotnull(sr_item_sk#9) AND isnotnull(sr_ticket_number#10)) -(8) CometProject +(7) CometProject Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -(9) ColumnarToRow [codegen id : 3] -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] - -(10) Exchange +(8) CometColumnarExchange Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(11) Sort [codegen id : 4] +(9) CometSort Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST], false, 0 +Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST] -(12) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_item_sk#1, ss_ticket_number#4] -Right keys [2]: [sr_item_sk#9, sr_ticket_number#10] -Join type: LeftOuter -Join condition: None +(10) CometSortMergeJoin +Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] +Right output [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] +Arguments: [ss_item_sk#1, ss_ticket_number#4], [sr_item_sk#9, sr_ticket_number#10], LeftOuter -(13) Project [codegen id : 9] -Output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] +(11) CometProject Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] +Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] -(14) ReusedExchange [Reuses operator id: 125] -Output [1]: [d_date_sk#14] +(12) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#15] +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 -(15) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None +(13) CometFilter +Input [2]: [d_date_sk#14, d_date#15] +Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) -(16) Project [codegen id : 9] -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] +(14) CometProject +Input [2]: [d_date_sk#14, d_date#15] +Arguments: [d_date_sk#14], [d_date_sk#14] + +(15) CometBroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: [d_date_sk#14] + +(16) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] +Right output [1]: [d_date_sk#14] +Arguments: [ss_sold_date_sk#7], [d_date_sk#14], Inner, BuildRight + +(17) CometProject Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#14] +Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] -(17) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#15, s_store_id#16] +(18) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#16, s_store_id#17] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(18) CometFilter -Input [2]: [s_store_sk#15, s_store_id#16] -Condition : isnotnull(s_store_sk#15) - -(19) ColumnarToRow [codegen id : 6] -Input [2]: [s_store_sk#15, s_store_id#16] +(19) CometFilter +Input [2]: [s_store_sk#16, s_store_id#17] +Condition : isnotnull(s_store_sk#16) -(20) BroadcastExchange -Input [2]: [s_store_sk#15, s_store_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(20) CometBroadcastExchange +Input [2]: [s_store_sk#16, s_store_id#17] +Arguments: [s_store_sk#16, s_store_id#17] -(21) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#15] -Join type: Inner -Join condition: None +(21) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] +Right output [2]: [s_store_sk#16, s_store_id#17] +Arguments: [ss_store_sk#2], [s_store_sk#16], Inner, BuildRight -(22) Project [codegen id : 9] -Output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#15, s_store_id#16] +(22) CometProject +Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#16, s_store_id#17] +Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17], [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] (23) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#17, i_current_price#18] +Output [2]: [i_item_sk#18, i_current_price#19] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), GreaterThan(i_current_price,50.00), IsNotNull(i_item_sk)] ReadSchema: struct (24) CometFilter -Input [2]: [i_item_sk#17, i_current_price#18] -Condition : ((isnotnull(i_current_price#18) AND (i_current_price#18 > 50.00)) AND isnotnull(i_item_sk#17)) +Input [2]: [i_item_sk#18, i_current_price#19] +Condition : ((isnotnull(i_current_price#19) AND (i_current_price#19 > 50.00)) AND isnotnull(i_item_sk#18)) (25) CometProject -Input [2]: [i_item_sk#17, i_current_price#18] -Arguments: [i_item_sk#17], [i_item_sk#17] +Input [2]: [i_item_sk#18, i_current_price#19] +Arguments: [i_item_sk#18], [i_item_sk#18] -(26) ColumnarToRow [codegen id : 7] -Input [1]: [i_item_sk#17] +(26) CometBroadcastExchange +Input [1]: [i_item_sk#18] +Arguments: [i_item_sk#18] -(27) BroadcastExchange -Input [1]: [i_item_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(27) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] +Right output [1]: [i_item_sk#18] +Arguments: [ss_item_sk#1], [i_item_sk#18], Inner, BuildRight -(28) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#17] -Join type: Inner -Join condition: None +(28) CometProject +Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17, i_item_sk#18] +Arguments: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17], [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] -(29) Project [codegen id : 9] -Output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16, i_item_sk#17] - -(30) Scan parquet spark_catalog.default.promotion -Output [2]: [p_promo_sk#19, p_channel_tv#20] +(29) Scan parquet spark_catalog.default.promotion +Output [2]: [p_promo_sk#20, p_channel_tv#21] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_channel_tv), EqualTo(p_channel_tv,N), IsNotNull(p_promo_sk)] ReadSchema: struct -(31) CometFilter -Input [2]: [p_promo_sk#19, p_channel_tv#20] -Condition : ((isnotnull(p_channel_tv#20) AND (p_channel_tv#20 = N)) AND isnotnull(p_promo_sk#19)) - -(32) CometProject -Input [2]: [p_promo_sk#19, p_channel_tv#20] -Arguments: [p_promo_sk#19], [p_promo_sk#19] +(30) CometFilter +Input [2]: [p_promo_sk#20, p_channel_tv#21] +Condition : ((isnotnull(p_channel_tv#21) AND (p_channel_tv#21 = N)) AND isnotnull(p_promo_sk#20)) -(33) ColumnarToRow [codegen id : 8] -Input [1]: [p_promo_sk#19] +(31) CometProject +Input [2]: [p_promo_sk#20, p_channel_tv#21] +Arguments: [p_promo_sk#20], [p_promo_sk#20] -(34) BroadcastExchange -Input [1]: [p_promo_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(32) CometBroadcastExchange +Input [1]: [p_promo_sk#20] +Arguments: [p_promo_sk#20] -(35) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_promo_sk#3] -Right keys [1]: [p_promo_sk#19] -Join type: Inner -Join condition: None +(33) CometBroadcastHashJoin +Left output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] +Right output [1]: [p_promo_sk#20] +Arguments: [ss_promo_sk#3], [p_promo_sk#20], Inner, BuildRight -(36) Project [codegen id : 9] -Output [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] -Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16, p_promo_sk#19] +(34) CometProject +Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17, p_promo_sk#20] +Arguments: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17], [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] -(37) HashAggregate [codegen id : 9] -Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] -Keys [1]: [s_store_id#16] +(35) CometHashAggregate +Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] +Keys [1]: [s_store_id#17] Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] -Aggregate Attributes [5]: [sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -Results [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] -(38) Exchange -Input [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] -Arguments: hashpartitioning(s_store_id#16, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(36) CometColumnarExchange +Input [6]: [s_store_id#17, sum#22, sum#23, isEmpty#24, sum#25, isEmpty#26] +Arguments: hashpartitioning(s_store_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(39) HashAggregate [codegen id : 10] -Input [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] -Keys [1]: [s_store_id#16] +(37) ColumnarToRow [codegen id : 1] +Input [6]: [s_store_id#17, sum#22, sum#23, isEmpty#24, sum#25, isEmpty#26] + +(38) HashAggregate [codegen id : 1] +Input [6]: [s_store_id#17, sum#22, sum#23, isEmpty#24, sum#25, isEmpty#26] +Keys [1]: [s_store_id#17] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#5))#31, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#32, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#33] -Results [5]: [store channel AS channel#34, concat(store, s_store_id#16) AS id#35, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#31,17,2) AS sales#36, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#32 AS returns#37, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#33 AS profit#38] +Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#5))#27, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#28, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#29] +Results [5]: [store channel AS channel#30, concat(store, s_store_id#17) AS id#31, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#27,17,2) AS sales#32, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#28 AS returns#33, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#29 AS profit#34] -(40) Scan parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +(39) Scan parquet spark_catalog.default.catalog_sales +Output [7]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_order_number#38, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41] 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#41), dynamicpruningexpression(cs_sold_date_sk#41 IN dynamicpruning#42)] PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] ReadSchema: struct -(41) CometFilter -Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Condition : ((isnotnull(cs_catalog_page_sk#39) AND isnotnull(cs_item_sk#40)) AND isnotnull(cs_promo_sk#41)) - -(42) ColumnarToRow [codegen id : 11] -Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +(40) CometFilter +Input [7]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_order_number#38, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41] +Condition : ((isnotnull(cs_catalog_page_sk#35) AND isnotnull(cs_item_sk#36)) AND isnotnull(cs_promo_sk#37)) -(43) Exchange -Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Arguments: hashpartitioning(cs_item_sk#40, cs_order_number#42, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(41) CometColumnarExchange +Input [7]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_order_number#38, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41] +Arguments: hashpartitioning(cs_item_sk#36, cs_order_number#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(44) Sort [codegen id : 12] -Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Arguments: [cs_item_sk#40 ASC NULLS FIRST, cs_order_number#42 ASC NULLS FIRST], false, 0 +(42) CometSort +Input [7]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_order_number#38, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41] +Arguments: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_order_number#38, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41], [cs_item_sk#36 ASC NULLS FIRST, cs_order_number#38 ASC NULLS FIRST] -(45) Scan parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] +(43) Scan parquet spark_catalog.default.catalog_returns +Output [5]: [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46, cr_returned_date_sk#47] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(46) CometFilter -Input [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] -Condition : (isnotnull(cr_item_sk#47) AND isnotnull(cr_order_number#48)) - -(47) CometProject -Input [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] -Arguments: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50], [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] +(44) CometFilter +Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46, cr_returned_date_sk#47] +Condition : (isnotnull(cr_item_sk#43) AND isnotnull(cr_order_number#44)) -(48) ColumnarToRow [codegen id : 13] -Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] +(45) CometProject +Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46, cr_returned_date_sk#47] +Arguments: [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46], [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46] -(49) Exchange -Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -Arguments: hashpartitioning(cr_item_sk#47, cr_order_number#48, 5), ENSURE_REQUIREMENTS, [plan_id=8] +(46) CometColumnarExchange +Input [4]: [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46] +Arguments: hashpartitioning(cr_item_sk#43, cr_order_number#44, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(50) Sort [codegen id : 14] -Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -Arguments: [cr_item_sk#47 ASC NULLS FIRST, cr_order_number#48 ASC NULLS FIRST], false, 0 +(47) CometSort +Input [4]: [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46] +Arguments: [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46], [cr_item_sk#43 ASC NULLS FIRST, cr_order_number#44 ASC NULLS FIRST] -(51) SortMergeJoin [codegen id : 19] -Left keys [2]: [cs_item_sk#40, cs_order_number#42] -Right keys [2]: [cr_item_sk#47, cr_order_number#48] -Join type: LeftOuter -Join condition: None +(48) CometSortMergeJoin +Left output [7]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_order_number#38, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41] +Right output [4]: [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46] +Arguments: [cs_item_sk#36, cs_order_number#38], [cr_item_sk#43, cr_order_number#44], LeftOuter -(52) Project [codegen id : 19] -Output [8]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50] -Input [11]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] +(49) CometProject +Input [11]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_order_number#38, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41, cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46] +Arguments: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41, cr_return_amount#45, cr_net_loss#46], [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41, cr_return_amount#45, cr_net_loss#46] -(53) ReusedExchange [Reuses operator id: 125] -Output [1]: [d_date_sk#52] +(50) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#48] -(54) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_sold_date_sk#45] -Right keys [1]: [d_date_sk#52] -Join type: Inner -Join condition: None +(51) CometBroadcastHashJoin +Left output [8]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41, cr_return_amount#45, cr_net_loss#46] +Right output [1]: [d_date_sk#48] +Arguments: [cs_sold_date_sk#41], [d_date_sk#48], Inner, BuildRight -(55) Project [codegen id : 19] -Output [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50] -Input [9]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50, d_date_sk#52] +(52) CometProject +Input [9]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41, cr_return_amount#45, cr_net_loss#46, d_date_sk#48] +Arguments: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46], [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46] -(56) Scan parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +(53) Scan parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#49, cp_catalog_page_id#50] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(57) CometFilter -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Condition : isnotnull(cp_catalog_page_sk#53) - -(58) ColumnarToRow [codegen id : 16] -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] - -(59) BroadcastExchange -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -(60) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_catalog_page_sk#39] -Right keys [1]: [cp_catalog_page_sk#53] -Join type: Inner -Join condition: None - -(61) Project [codegen id : 19] -Output [7]: [cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] -Input [9]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_sk#53, cp_catalog_page_id#54] - -(62) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#55] - -(63) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_item_sk#40] -Right keys [1]: [i_item_sk#55] -Join type: Inner -Join condition: None - -(64) Project [codegen id : 19] -Output [6]: [cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] -Input [8]: [cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54, i_item_sk#55] - -(65) ReusedExchange [Reuses operator id: 34] -Output [1]: [p_promo_sk#56] - -(66) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_promo_sk#41] -Right keys [1]: [p_promo_sk#56] -Join type: Inner -Join condition: None - -(67) Project [codegen id : 19] -Output [5]: [cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] -Input [7]: [cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54, p_promo_sk#56] - -(68) HashAggregate [codegen id : 19] -Input [5]: [cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] -Keys [1]: [cp_catalog_page_id#54] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#43)), partial_sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))] -Aggregate Attributes [5]: [sum#57, sum#58, isEmpty#59, sum#60, isEmpty#61] -Results [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] - -(69) Exchange -Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] -Arguments: hashpartitioning(cp_catalog_page_id#54, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(70) HashAggregate [codegen id : 20] -Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] -Keys [1]: [cp_catalog_page_id#54] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#43)), sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00)), sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#43))#67, sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00))#68, sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))#69] -Results [5]: [catalog channel AS channel#70, concat(catalog_page, cp_catalog_page_id#54) AS id#71, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#43))#67,17,2) AS sales#72, sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00))#68 AS returns#73, sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))#69 AS profit#74] - -(71) Scan parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] +(54) CometFilter +Input [2]: [cp_catalog_page_sk#49, cp_catalog_page_id#50] +Condition : isnotnull(cp_catalog_page_sk#49) + +(55) CometBroadcastExchange +Input [2]: [cp_catalog_page_sk#49, cp_catalog_page_id#50] +Arguments: [cp_catalog_page_sk#49, cp_catalog_page_id#50] + +(56) CometBroadcastHashJoin +Left output [7]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46] +Right output [2]: [cp_catalog_page_sk#49, cp_catalog_page_id#50] +Arguments: [cs_catalog_page_sk#35], [cp_catalog_page_sk#49], Inner, BuildRight + +(57) CometProject +Input [9]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_sk#49, cp_catalog_page_id#50] +Arguments: [cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50], [cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50] + +(58) ReusedExchange [Reuses operator id: 26] +Output [1]: [i_item_sk#51] + +(59) CometBroadcastHashJoin +Left output [7]: [cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50] +Right output [1]: [i_item_sk#51] +Arguments: [cs_item_sk#36], [i_item_sk#51], Inner, BuildRight + +(60) CometProject +Input [8]: [cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50, i_item_sk#51] +Arguments: [cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50], [cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50] + +(61) ReusedExchange [Reuses operator id: 32] +Output [1]: [p_promo_sk#52] + +(62) CometBroadcastHashJoin +Left output [6]: [cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50] +Right output [1]: [p_promo_sk#52] +Arguments: [cs_promo_sk#37], [p_promo_sk#52], Inner, BuildRight + +(63) CometProject +Input [7]: [cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50, p_promo_sk#52] +Arguments: [cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50], [cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50] + +(64) CometHashAggregate +Input [5]: [cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50] +Keys [1]: [cp_catalog_page_id#50] +Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#39)), partial_sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#40 - coalesce(cast(cr_net_loss#46 as decimal(12,2)), 0.00)))] + +(65) CometColumnarExchange +Input [6]: [cp_catalog_page_id#50, sum#53, sum#54, isEmpty#55, sum#56, isEmpty#57] +Arguments: hashpartitioning(cp_catalog_page_id#50, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(66) ColumnarToRow [codegen id : 2] +Input [6]: [cp_catalog_page_id#50, sum#53, sum#54, isEmpty#55, sum#56, isEmpty#57] + +(67) HashAggregate [codegen id : 2] +Input [6]: [cp_catalog_page_id#50, sum#53, sum#54, isEmpty#55, sum#56, isEmpty#57] +Keys [1]: [cp_catalog_page_id#50] +Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#39)), sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00)), sum((cs_net_profit#40 - coalesce(cast(cr_net_loss#46 as decimal(12,2)), 0.00)))] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#39))#58, sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00))#59, sum((cs_net_profit#40 - coalesce(cast(cr_net_loss#46 as decimal(12,2)), 0.00)))#60] +Results [5]: [catalog channel AS channel#61, concat(catalog_page, cp_catalog_page_id#50) AS id#62, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#39))#58,17,2) AS sales#63, sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00))#59 AS returns#64, sum((cs_net_profit#40 - coalesce(cast(cr_net_loss#46 as decimal(12,2)), 0.00)))#60 AS profit#65] + +(68) Scan parquet spark_catalog.default.web_sales +Output [7]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_order_number#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] 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#72), dynamicpruningexpression(ws_sold_date_sk#72 IN dynamicpruning#73)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct -(72) CometFilter -Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Condition : ((isnotnull(ws_web_site_sk#76) AND isnotnull(ws_item_sk#75)) AND isnotnull(ws_promo_sk#77)) - -(73) ColumnarToRow [codegen id : 21] -Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] +(69) CometFilter +Input [7]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_order_number#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Condition : ((isnotnull(ws_web_site_sk#67) AND isnotnull(ws_item_sk#66)) AND isnotnull(ws_promo_sk#68)) -(74) Exchange -Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Arguments: hashpartitioning(ws_item_sk#75, ws_order_number#78, 5), ENSURE_REQUIREMENTS, [plan_id=11] +(70) CometColumnarExchange +Input [7]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_order_number#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Arguments: hashpartitioning(ws_item_sk#66, ws_order_number#69, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(75) Sort [codegen id : 22] -Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Arguments: [ws_item_sk#75 ASC NULLS FIRST, ws_order_number#78 ASC NULLS FIRST], false, 0 +(71) CometSort +Input [7]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_order_number#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Arguments: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_order_number#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72], [ws_item_sk#66 ASC NULLS FIRST, ws_order_number#69 ASC NULLS FIRST] -(76) Scan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +(72) Scan parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77, wr_returned_date_sk#78] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct -(77) CometFilter -Input [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] -Condition : (isnotnull(wr_item_sk#83) AND isnotnull(wr_order_number#84)) +(73) CometFilter +Input [5]: [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77, wr_returned_date_sk#78] +Condition : (isnotnull(wr_item_sk#74) AND isnotnull(wr_order_number#75)) -(78) CometProject -Input [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] -Arguments: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86], [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] +(74) CometProject +Input [5]: [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77, wr_returned_date_sk#78] +Arguments: [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77], [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77] -(79) ColumnarToRow [codegen id : 23] -Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] +(75) CometColumnarExchange +Input [4]: [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77] +Arguments: hashpartitioning(wr_item_sk#74, wr_order_number#75, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(80) Exchange -Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -Arguments: hashpartitioning(wr_item_sk#83, wr_order_number#84, 5), ENSURE_REQUIREMENTS, [plan_id=12] +(76) CometSort +Input [4]: [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77] +Arguments: [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77], [wr_item_sk#74 ASC NULLS FIRST, wr_order_number#75 ASC NULLS FIRST] -(81) Sort [codegen id : 24] -Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -Arguments: [wr_item_sk#83 ASC NULLS FIRST, wr_order_number#84 ASC NULLS FIRST], false, 0 +(77) CometSortMergeJoin +Left output [7]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_order_number#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Right output [4]: [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77] +Arguments: [ws_item_sk#66, ws_order_number#69], [wr_item_sk#74, wr_order_number#75], LeftOuter -(82) SortMergeJoin [codegen id : 29] -Left keys [2]: [ws_item_sk#75, ws_order_number#78] -Right keys [2]: [wr_item_sk#83, wr_order_number#84] -Join type: LeftOuter -Join condition: None - -(83) Project [codegen id : 29] -Output [8]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86] -Input [11]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] +(78) CometProject +Input [11]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_order_number#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77] +Arguments: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, wr_return_amt#76, wr_net_loss#77], [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, wr_return_amt#76, wr_net_loss#77] -(84) ReusedExchange [Reuses operator id: 125] -Output [1]: [d_date_sk#88] +(79) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#79] -(85) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_sold_date_sk#81] -Right keys [1]: [d_date_sk#88] -Join type: Inner -Join condition: None +(80) CometBroadcastHashJoin +Left output [8]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, wr_return_amt#76, wr_net_loss#77] +Right output [1]: [d_date_sk#79] +Arguments: [ws_sold_date_sk#72], [d_date_sk#79], Inner, BuildRight -(86) Project [codegen id : 29] -Output [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86] -Input [9]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86, d_date_sk#88] +(81) CometProject +Input [9]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, wr_return_amt#76, wr_net_loss#77, d_date_sk#79] +Arguments: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77], [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77] -(87) Scan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#89, web_site_id#90] +(82) Scan parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#80, web_site_id#81] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(88) CometFilter -Input [2]: [web_site_sk#89, web_site_id#90] -Condition : isnotnull(web_site_sk#89) - -(89) ColumnarToRow [codegen id : 26] -Input [2]: [web_site_sk#89, web_site_id#90] - -(90) BroadcastExchange -Input [2]: [web_site_sk#89, web_site_id#90] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] - -(91) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_web_site_sk#76] -Right keys [1]: [web_site_sk#89] -Join type: Inner -Join condition: None - -(92) Project [codegen id : 29] -Output [7]: [ws_item_sk#75, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] -Input [9]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_sk#89, web_site_id#90] - -(93) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#91] - -(94) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_item_sk#75] -Right keys [1]: [i_item_sk#91] -Join type: Inner -Join condition: None - -(95) Project [codegen id : 29] -Output [6]: [ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] -Input [8]: [ws_item_sk#75, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90, i_item_sk#91] - -(96) ReusedExchange [Reuses operator id: 34] -Output [1]: [p_promo_sk#92] - -(97) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_promo_sk#77] -Right keys [1]: [p_promo_sk#92] -Join type: Inner -Join condition: None - -(98) Project [codegen id : 29] -Output [5]: [ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] -Input [7]: [ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90, p_promo_sk#92] - -(99) HashAggregate [codegen id : 29] -Input [5]: [ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] -Keys [1]: [web_site_id#90] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#79)), partial_sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))] -Aggregate Attributes [5]: [sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] -Results [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] - -(100) Exchange -Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -Arguments: hashpartitioning(web_site_id#90, 5), ENSURE_REQUIREMENTS, [plan_id=14] - -(101) HashAggregate [codegen id : 30] -Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -Keys [1]: [web_site_id#90] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#79)), sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00)), sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#79))#103, sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00))#104, sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))#105] -Results [5]: [web channel AS channel#106, concat(web_site, web_site_id#90) AS id#107, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#79))#103,17,2) AS sales#108, sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00))#104 AS returns#109, sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))#105 AS profit#110] - -(102) Union - -(103) HashAggregate [codegen id : 31] -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#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116] -Results [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] - -(104) Exchange -Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] -Arguments: hashpartitioning(channel#34, id#35, 5), ENSURE_REQUIREMENTS, [plan_id=15] - -(105) HashAggregate [codegen id : 32] -Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] -Keys [2]: [channel#34, id#35] -Functions [3]: [sum(sales#36), sum(returns#37), sum(profit#38)] -Aggregate Attributes [3]: [sum(sales#36)#123, sum(returns#37)#124, sum(profit#38)#125] -Results [5]: [channel#34, id#35, cast(sum(sales#36)#123 as decimal(37,2)) AS sales#126, cast(sum(returns#37)#124 as decimal(38,2)) AS returns#127, cast(sum(profit#38)#125 as decimal(38,2)) AS profit#128] - -(106) ReusedExchange [Reuses operator id: 104] -Output [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] - -(107) HashAggregate [codegen id : 64] -Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] -Keys [2]: [channel#34, id#35] -Functions [3]: [sum(sales#36), sum(returns#37), sum(profit#38)] -Aggregate Attributes [3]: [sum(sales#36)#123, sum(returns#37)#124, sum(profit#38)#125] -Results [4]: [channel#34, sum(sales#36)#123 AS sales#129, sum(returns#37)#124 AS returns#130, sum(profit#38)#125 AS profit#131] - -(108) HashAggregate [codegen id : 64] -Input [4]: [channel#34, sales#129, returns#130, profit#131] -Keys [1]: [channel#34] -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#34, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] - -(109) Exchange -Input [7]: [channel#34, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] -Arguments: hashpartitioning(channel#34, 5), ENSURE_REQUIREMENTS, [plan_id=16] - -(110) HashAggregate [codegen id : 65] -Input [7]: [channel#34, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] -Keys [1]: [channel#34] -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#34, 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] - -(111) ReusedExchange [Reuses operator id: 104] -Output [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] - -(112) HashAggregate [codegen id : 97] -Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] -Keys [2]: [channel#34, id#35] -Functions [3]: [sum(sales#36), sum(returns#37), sum(profit#38)] -Aggregate Attributes [3]: [sum(sales#36)#123, sum(returns#37)#124, sum(profit#38)#125] -Results [3]: [sum(sales#36)#123 AS sales#151, sum(returns#37)#124 AS returns#152, sum(profit#38)#125 AS profit#153] - -(113) HashAggregate [codegen id : 97] -Input [3]: [sales#151, returns#152, profit#153] +(83) CometFilter +Input [2]: [web_site_sk#80, web_site_id#81] +Condition : isnotnull(web_site_sk#80) + +(84) CometBroadcastExchange +Input [2]: [web_site_sk#80, web_site_id#81] +Arguments: [web_site_sk#80, web_site_id#81] + +(85) CometBroadcastHashJoin +Left output [7]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77] +Right output [2]: [web_site_sk#80, web_site_id#81] +Arguments: [ws_web_site_sk#67], [web_site_sk#80], Inner, BuildRight + +(86) CometProject +Input [9]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_sk#80, web_site_id#81] +Arguments: [ws_item_sk#66, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81], [ws_item_sk#66, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81] + +(87) ReusedExchange [Reuses operator id: 26] +Output [1]: [i_item_sk#82] + +(88) CometBroadcastHashJoin +Left output [7]: [ws_item_sk#66, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81] +Right output [1]: [i_item_sk#82] +Arguments: [ws_item_sk#66], [i_item_sk#82], Inner, BuildRight + +(89) CometProject +Input [8]: [ws_item_sk#66, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81, i_item_sk#82] +Arguments: [ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81], [ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81] + +(90) ReusedExchange [Reuses operator id: 32] +Output [1]: [p_promo_sk#83] + +(91) CometBroadcastHashJoin +Left output [6]: [ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81] +Right output [1]: [p_promo_sk#83] +Arguments: [ws_promo_sk#68], [p_promo_sk#83], Inner, BuildRight + +(92) CometProject +Input [7]: [ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81, p_promo_sk#83] +Arguments: [ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81], [ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81] + +(93) CometHashAggregate +Input [5]: [ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81] +Keys [1]: [web_site_id#81] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(coalesce(cast(wr_return_amt#76 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#71 - coalesce(cast(wr_net_loss#77 as decimal(12,2)), 0.00)))] + +(94) CometColumnarExchange +Input [6]: [web_site_id#81, sum#84, sum#85, isEmpty#86, sum#87, isEmpty#88] +Arguments: hashpartitioning(web_site_id#81, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(95) ColumnarToRow [codegen id : 3] +Input [6]: [web_site_id#81, sum#84, sum#85, isEmpty#86, sum#87, isEmpty#88] + +(96) HashAggregate [codegen id : 3] +Input [6]: [web_site_id#81, sum#84, sum#85, isEmpty#86, sum#87, isEmpty#88] +Keys [1]: [web_site_id#81] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#70)), sum(coalesce(cast(wr_return_amt#76 as decimal(12,2)), 0.00)), sum((ws_net_profit#71 - coalesce(cast(wr_net_loss#77 as decimal(12,2)), 0.00)))] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#70))#89, sum(coalesce(cast(wr_return_amt#76 as decimal(12,2)), 0.00))#90, sum((ws_net_profit#71 - coalesce(cast(wr_net_loss#77 as decimal(12,2)), 0.00)))#91] +Results [5]: [web channel AS channel#92, concat(web_site, web_site_id#81) AS id#93, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#70))#89,17,2) AS sales#94, sum(coalesce(cast(wr_return_amt#76 as decimal(12,2)), 0.00))#90 AS returns#95, sum((ws_net_profit#71 - coalesce(cast(wr_net_loss#77 as decimal(12,2)), 0.00)))#91 AS profit#96] + +(97) Union + +(98) HashAggregate [codegen id : 4] +Input [5]: [channel#30, id#31, sales#32, returns#33, profit#34] +Keys [2]: [channel#30, id#31] +Functions [3]: [partial_sum(sales#32), partial_sum(returns#33), partial_sum(profit#34)] +Aggregate Attributes [6]: [sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102] +Results [8]: [channel#30, id#31, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] + +(99) RowToColumnar +Input [8]: [channel#30, id#31, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] + +(100) CometColumnarExchange +Input [8]: [channel#30, id#31, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] +Arguments: hashpartitioning(channel#30, id#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(101) ColumnarToRow [codegen id : 5] +Input [8]: [channel#30, id#31, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] + +(102) HashAggregate [codegen id : 5] +Input [8]: [channel#30, id#31, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] +Keys [2]: [channel#30, id#31] +Functions [3]: [sum(sales#32), sum(returns#33), sum(profit#34)] +Aggregate Attributes [3]: [sum(sales#32)#109, sum(returns#33)#110, sum(profit#34)#111] +Results [5]: [channel#30, id#31, cast(sum(sales#32)#109 as decimal(37,2)) AS sales#112, cast(sum(returns#33)#110 as decimal(38,2)) AS returns#113, cast(sum(profit#34)#111 as decimal(38,2)) AS profit#114] + +(103) ReusedExchange [Reuses operator id: 100] +Output [8]: [channel#30, id#31, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] + +(104) ColumnarToRow [codegen id : 10] +Input [8]: [channel#30, id#31, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] + +(105) HashAggregate [codegen id : 10] +Input [8]: [channel#30, id#31, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] +Keys [2]: [channel#30, id#31] +Functions [3]: [sum(sales#32), sum(returns#33), sum(profit#34)] +Aggregate Attributes [3]: [sum(sales#32)#109, sum(returns#33)#110, sum(profit#34)#111] +Results [4]: [channel#30, sum(sales#32)#109 AS sales#115, sum(returns#33)#110 AS returns#116, sum(profit#34)#111 AS profit#117] + +(106) HashAggregate [codegen id : 10] +Input [4]: [channel#30, sales#115, returns#116, profit#117] +Keys [1]: [channel#30] +Functions [3]: [partial_sum(sales#115), partial_sum(returns#116), partial_sum(profit#117)] +Aggregate Attributes [6]: [sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123] +Results [7]: [channel#30, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] + +(107) RowToColumnar +Input [7]: [channel#30, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] + +(108) CometColumnarExchange +Input [7]: [channel#30, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] +Arguments: hashpartitioning(channel#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] + +(109) ColumnarToRow [codegen id : 11] +Input [7]: [channel#30, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] + +(110) HashAggregate [codegen id : 11] +Input [7]: [channel#30, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] +Keys [1]: [channel#30] +Functions [3]: [sum(sales#115), sum(returns#116), sum(profit#117)] +Aggregate Attributes [3]: [sum(sales#115)#130, sum(returns#116)#131, sum(profit#117)#132] +Results [5]: [channel#30, null AS id#133, sum(sales#115)#130 AS sales#134, sum(returns#116)#131 AS returns#135, sum(profit#117)#132 AS profit#136] + +(111) ReusedExchange [Reuses operator id: 100] +Output [8]: [channel#30, id#31, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] + +(112) ColumnarToRow [codegen id : 16] +Input [8]: [channel#30, id#31, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] + +(113) HashAggregate [codegen id : 16] +Input [8]: [channel#30, id#31, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] +Keys [2]: [channel#30, id#31] +Functions [3]: [sum(sales#32), sum(returns#33), sum(profit#34)] +Aggregate Attributes [3]: [sum(sales#32)#109, sum(returns#33)#110, sum(profit#34)#111] +Results [3]: [sum(sales#32)#109 AS sales#137, sum(returns#33)#110 AS returns#138, sum(profit#34)#111 AS profit#139] + +(114) HashAggregate [codegen id : 16] +Input [3]: [sales#137, returns#138, profit#139] Keys: [] -Functions [3]: [partial_sum(sales#151), partial_sum(returns#152), partial_sum(profit#153)] -Aggregate Attributes [6]: [sum#154, isEmpty#155, sum#156, isEmpty#157, sum#158, isEmpty#159] -Results [6]: [sum#160, isEmpty#161, sum#162, isEmpty#163, sum#164, isEmpty#165] +Functions [3]: [partial_sum(sales#137), partial_sum(returns#138), partial_sum(profit#139)] +Aggregate Attributes [6]: [sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] +Results [6]: [sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] -(114) Exchange -Input [6]: [sum#160, isEmpty#161, sum#162, isEmpty#163, sum#164, isEmpty#165] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=17] +(115) RowToColumnar +Input [6]: [sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] -(115) HashAggregate [codegen id : 98] -Input [6]: [sum#160, isEmpty#161, sum#162, isEmpty#163, sum#164, isEmpty#165] +(116) CometColumnarExchange +Input [6]: [sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] + +(117) ColumnarToRow [codegen id : 17] +Input [6]: [sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] + +(118) HashAggregate [codegen id : 17] +Input [6]: [sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] Keys: [] -Functions [3]: [sum(sales#151), sum(returns#152), sum(profit#153)] -Aggregate Attributes [3]: [sum(sales#151)#166, sum(returns#152)#167, sum(profit#153)#168] -Results [5]: [null AS channel#169, null AS id#170, sum(sales#151)#166 AS sales#171, sum(returns#152)#167 AS returns#172, sum(profit#153)#168 AS profit#173] +Functions [3]: [sum(sales#137), sum(returns#138), sum(profit#139)] +Aggregate Attributes [3]: [sum(sales#137)#152, sum(returns#138)#153, sum(profit#139)#154] +Results [5]: [null AS channel#155, null AS id#156, sum(sales#137)#152 AS sales#157, sum(returns#138)#153 AS returns#158, sum(profit#139)#154 AS profit#159] -(116) Union +(119) Union -(117) HashAggregate [codegen id : 99] -Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] -Keys [5]: [channel#34, id#35, sales#126, returns#127, profit#128] +(120) HashAggregate [codegen id : 18] +Input [5]: [channel#30, id#31, sales#112, returns#113, profit#114] +Keys [5]: [channel#30, id#31, sales#112, returns#113, profit#114] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#34, id#35, sales#126, returns#127, profit#128] +Results [5]: [channel#30, id#31, sales#112, returns#113, profit#114] + +(121) RowToColumnar +Input [5]: [channel#30, id#31, sales#112, returns#113, profit#114] -(118) Exchange -Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] -Arguments: hashpartitioning(channel#34, id#35, sales#126, returns#127, profit#128, 5), ENSURE_REQUIREMENTS, [plan_id=18] +(122) CometColumnarExchange +Input [5]: [channel#30, id#31, sales#112, returns#113, profit#114] +Arguments: hashpartitioning(channel#30, id#31, sales#112, returns#113, profit#114, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(119) HashAggregate [codegen id : 100] -Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] -Keys [5]: [channel#34, id#35, sales#126, returns#127, profit#128] +(123) CometHashAggregate +Input [5]: [channel#30, id#31, sales#112, returns#113, profit#114] +Keys [5]: [channel#30, id#31, sales#112, returns#113, profit#114] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#34, id#35, sales#126, returns#127, profit#128] -(120) TakeOrderedAndProject -Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] -Arguments: 100, [channel#34 ASC NULLS FIRST, id#35 ASC NULLS FIRST], [channel#34, id#35, sales#126, returns#127, profit#128] +(124) CometTakeOrderedAndProject +Input [5]: [channel#30, id#31, sales#112, returns#113, profit#114] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#30 ASC NULLS FIRST,id#31 ASC NULLS FIRST], output=[channel#30,id#31,sales#112,returns#113,profit#114]), [channel#30, id#31, sales#112, returns#113, profit#114], 100, [channel#30 ASC NULLS FIRST, id#31 ASC NULLS FIRST], [channel#30, id#31, sales#112, returns#113, profit#114] + +(125) ColumnarToRow [codegen id : 19] +Input [5]: [channel#30, id#31, sales#112, returns#113, profit#114] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (125) -+- * ColumnarToRow (124) - +- CometProject (123) - +- CometFilter (122) - +- CometScan parquet spark_catalog.default.date_dim (121) +BroadcastExchange (130) ++- * ColumnarToRow (129) + +- CometProject (128) + +- CometFilter (127) + +- CometScan parquet spark_catalog.default.date_dim (126) -(121) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#174] +(126) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#15] 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 -(122) CometFilter -Input [2]: [d_date_sk#14, d_date#174] -Condition : (((isnotnull(d_date#174) AND (d_date#174 >= 1998-08-04)) AND (d_date#174 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) +(127) CometFilter +Input [2]: [d_date_sk#14, d_date#15] +Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) -(123) CometProject -Input [2]: [d_date_sk#14, d_date#174] +(128) CometProject +Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(124) ColumnarToRow [codegen id : 1] +(129) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(125) BroadcastExchange +(130) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] -Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 71 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#72 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt index 0793067df..210c9b6d2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt @@ -1,207 +1,157 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (100) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Exchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (99) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (32) - 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 (31) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (10) - HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] +WholeStageCodegen (19) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [channel,id,sales,returns,profit] + CometColumnarExchange [channel,id,sales,returns,profit] #1 + RowToColumnar + WholeStageCodegen (18) + HashAggregate [channel,id,sales,returns,profit] + InputAdapter + Union + WholeStageCodegen (5) + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel,id] #2 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange [s_store_id] #3 - WholeStageCodegen (9) - HashAggregate [s_store_id,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - SortMergeJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - WholeStageCodegen (2) - Sort [ss_item_sk,ss_ticket_number] - InputAdapter - Exchange [ss_item_sk,ss_ticket_number] #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - WholeStageCodegen (4) - Sort [sr_item_sk,sr_ticket_number] - InputAdapter - Exchange [sr_item_sk,sr_ticket_number] #6 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter + Union + WholeStageCodegen (1) + HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [s_store_id] #3 + CometHashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] + CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] + CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] + CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] + CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk,ss_ticket_number] #4 + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometColumnarExchange [sr_item_sk,sr_ticket_number] #6 + CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #7 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [s_store_sk,s_store_id] #8 CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter + CometBroadcastExchange [i_item_sk] #9 CometProject [i_item_sk] CometFilter [i_item_sk,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter + CometBroadcastExchange [p_promo_sk] #10 CometProject [p_promo_sk] CometFilter [p_promo_sk,p_channel_tv] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - WholeStageCodegen (20) - HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [cp_catalog_page_id] #10 - WholeStageCodegen (19) - HashAggregate [cp_catalog_page_id,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_catalog_page_sk,cp_catalog_page_sk] - Project [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] - InputAdapter - WholeStageCodegen (12) - Sort [cs_item_sk,cs_order_number] - InputAdapter - Exchange [cs_item_sk,cs_order_number] #11 - WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - WholeStageCodegen (14) - Sort [cr_item_sk,cr_order_number] - InputAdapter - Exchange [cr_item_sk,cr_order_number] #12 - WholeStageCodegen (13) - ColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (16) - ColumnarToRow - InputAdapter + WholeStageCodegen (2) + HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [cp_catalog_page_id] #11 + CometHashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] + CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] + CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] + CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] + CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometColumnarExchange [cs_item_sk,cs_order_number] #12 + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometColumnarExchange [cr_item_sk,cr_order_number] #13 + CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #14 CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - InputAdapter - ReusedExchange [i_item_sk] #8 - InputAdapter - ReusedExchange [p_promo_sk] #9 - WholeStageCodegen (30) - HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [web_site_id] #14 - WholeStageCodegen (29) - HashAggregate [web_site_id,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_promo_sk,p_promo_sk] - Project [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - SortMergeJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] - InputAdapter - WholeStageCodegen (22) - Sort [ws_item_sk,ws_order_number] - InputAdapter - Exchange [ws_item_sk,ws_order_number] #15 - WholeStageCodegen (21) - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - WholeStageCodegen (24) - Sort [wr_item_sk,wr_order_number] - InputAdapter - Exchange [wr_item_sk,wr_order_number] #16 - WholeStageCodegen (23) - ColumnarToRow - InputAdapter - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (26) - ColumnarToRow - InputAdapter + ReusedExchange [i_item_sk] #9 + ReusedExchange [p_promo_sk] #10 + WholeStageCodegen (3) + HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [web_site_id] #15 + CometHashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] + CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] + CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] + CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] + CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometColumnarExchange [ws_item_sk,ws_order_number] #16 + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometColumnarExchange [wr_item_sk,wr_order_number] #17 + CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange [web_site_sk,web_site_id] #18 CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - InputAdapter - ReusedExchange [i_item_sk] #8 - InputAdapter - ReusedExchange [p_promo_sk] #9 - WholeStageCodegen (65) - 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] #18 - WholeStageCodegen (64) - 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 (98) - 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 #19 - WholeStageCodegen (97) - 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 - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + ReusedExchange [i_item_sk] #9 + ReusedExchange [p_promo_sk] #10 + WholeStageCodegen (11) + HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel] #19 + RowToColumnar + WholeStageCodegen (10) + 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] + ColumnarToRow + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + WholeStageCodegen (17) + HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange #20 + RowToColumnar + WholeStageCodegen (16) + 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] + ColumnarToRow + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/explain.txt index 93a322120..c2c6f352b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/explain.txt @@ -2,39 +2,39 @@ 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) + +- * ColumnarToRow (33) + +- CometSort (32) + +- CometColumnarExchange (31) + +- CometHashAggregate (30) + +- CometColumnarExchange (29) + +- CometHashAggregate (28) + +- CometUnion (27) + :- CometHashAggregate (16) + : +- CometColumnarExchange (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) + :- CometHashAggregate (21) + : +- CometColumnarExchange (20) + : +- CometHashAggregate (19) + : +- CometHashAggregate (18) + : +- ReusedExchange (17) + +- CometHashAggregate (26) + +- CometColumnarExchange (25) + +- CometHashAggregate (24) + +- CometHashAggregate (23) + +- ReusedExchange (22) (1) Scan parquet spark_catalog.default.web_sales @@ -106,115 +106,100 @@ 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))] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarExchange Input [3]: [i_category#9, i_class#8, sum#10] +Arguments: hashpartitioning(i_category#9, i_class#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(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] +(16) CometHashAggregate 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#9, i_class#8, 0 AS g_category#13, 0 AS g_class#14, 0 AS lochierarchy#15] - -(18) ReusedExchange [Reuses operator id: 16] -Output [3]: [i_category#16, i_class#17, sum#18] - -(19) HashAggregate [codegen id : 4] -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] - -(20) HashAggregate [codegen id : 4] -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] - -(21) Exchange -Input [3]: [i_category#16, sum#23, isEmpty#24] -Arguments: hashpartitioning(i_category#16, 5), ENSURE_REQUIREMENTS, [plan_id=2] - -(22) HashAggregate [codegen id : 5] -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] - -(23) ReusedExchange [Reuses operator id: 16] -Output [3]: [i_category#31, i_class#32, sum#33] - -(24) HashAggregate [codegen id : 7] -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] - -(25) HashAggregate [codegen id : 7] -Input [1]: [total_sum#35] + +(17) ReusedExchange [Reuses operator id: 15] +Output [3]: [i_category#11, i_class#12, sum#13] + +(18) CometHashAggregate +Input [3]: [i_category#11, i_class#12, sum#13] +Keys [2]: [i_category#11, i_class#12] +Functions [1]: [sum(UnscaledValue(ws_net_paid#14))] + +(19) CometHashAggregate +Input [2]: [total_sum#15, i_category#11] +Keys [1]: [i_category#11] +Functions [1]: [partial_sum(total_sum#15)] + +(20) CometColumnarExchange +Input [3]: [i_category#11, sum#16, isEmpty#17] +Arguments: hashpartitioning(i_category#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(21) CometHashAggregate +Input [3]: [i_category#11, sum#16, isEmpty#17] +Keys [1]: [i_category#11] +Functions [1]: [sum(total_sum#15)] + +(22) ReusedExchange [Reuses operator id: 15] +Output [3]: [i_category#18, i_class#19, sum#20] + +(23) CometHashAggregate +Input [3]: [i_category#18, i_class#19, sum#20] +Keys [2]: [i_category#18, i_class#19] +Functions [1]: [sum(UnscaledValue(ws_net_paid#21))] + +(24) CometHashAggregate +Input [1]: [total_sum#22] 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#22)] -(26) Exchange -Input [2]: [sum#38, isEmpty#39] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] +(25) CometColumnarExchange +Input [2]: [sum#23, isEmpty#24] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(27) HashAggregate [codegen id : 8] -Input [2]: [sum#38, isEmpty#39] +(26) CometHashAggregate +Input [2]: [sum#23, isEmpty#24] 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#22)] -(28) Union +(27) CometUnion +Child 0 Input [6]: [total_sum#25, i_category#9, i_class#8, g_category#26, g_class#27, lochierarchy#28] +Child 1 Input [6]: [total_sum#29, i_category#11, i_class#30, g_category#31, g_class#32, lochierarchy#33] +Child 2 Input [6]: [total_sum#34, i_category#35, i_class#36, g_category#37, g_class#38, lochierarchy#39] -(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] +(28) CometHashAggregate +Input [6]: [total_sum#25, i_category#9, i_class#8, g_category#26, g_class#27, lochierarchy#28] +Keys [6]: [total_sum#25, i_category#9, i_class#8, g_category#26, g_class#27, lochierarchy#28] Functions: [] -Aggregate Attributes: [] -Results [6]: [total_sum#12, i_category#9, i_class#8, g_category#13, g_class#14, lochierarchy#15] -(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) CometColumnarExchange +Input [6]: [total_sum#25, i_category#9, i_class#8, g_category#26, g_class#27, lochierarchy#28] +Arguments: hashpartitioning(total_sum#25, i_category#9, i_class#8, g_category#26, g_class#27, lochierarchy#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(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] +(30) CometHashAggregate +Input [6]: [total_sum#25, i_category#9, i_class#8, g_category#26, g_class#27, lochierarchy#28] +Keys [6]: [total_sum#25, i_category#9, i_class#8, g_category#26, g_class#27, lochierarchy#28] Functions: [] -Aggregate Attributes: [] -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#47] -(32) Exchange -Input [5]: [total_sum#12, i_category#9, i_class#8, lochierarchy#15, _w0#47] -Arguments: hashpartitioning(lochierarchy#15, _w0#47, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(31) CometColumnarExchange +Input [5]: [total_sum#25, i_category#9, i_class#8, lochierarchy#28, _w0#40] +Arguments: hashpartitioning(lochierarchy#28, _w0#40, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(32) CometSort +Input [5]: [total_sum#25, i_category#9, i_class#8, lochierarchy#28, _w0#40] +Arguments: [total_sum#25, i_category#9, i_class#8, lochierarchy#28, _w0#40], [lochierarchy#28 ASC NULLS FIRST, _w0#40 ASC NULLS FIRST, total_sum#25 DESC NULLS LAST] -(33) Sort [codegen id : 11] -Input [5]: [total_sum#12, i_category#9, i_class#8, lochierarchy#15, _w0#47] -Arguments: [lochierarchy#15 ASC NULLS FIRST, _w0#47 ASC NULLS FIRST, total_sum#12 DESC NULLS LAST], false, 0 +(33) ColumnarToRow [codegen id : 1] +Input [5]: [total_sum#25, i_category#9, i_class#8, lochierarchy#28, _w0#40] (34) Window -Input [5]: [total_sum#12, i_category#9, i_class#8, 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] +Input [5]: [total_sum#25, i_category#9, i_class#8, lochierarchy#28, _w0#40] +Arguments: [rank(total_sum#25) windowspecdefinition(lochierarchy#28, _w0#40, total_sum#25 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#41], [lochierarchy#28, _w0#40], [total_sum#25 DESC NULLS LAST] -(35) Project [codegen id : 12] -Output [5]: [total_sum#12, i_category#9, i_class#8, lochierarchy#15, rank_within_parent#48] -Input [6]: [total_sum#12, i_category#9, i_class#8, lochierarchy#15, _w0#47, rank_within_parent#48] +(35) Project [codegen id : 2] +Output [5]: [total_sum#25, i_category#9, i_class#8, lochierarchy#28, rank_within_parent#41] +Input [6]: [total_sum#25, i_category#9, i_class#8, lochierarchy#28, _w0#40, rank_within_parent#41] (36) TakeOrderedAndProject -Input [5]: [total_sum#12, i_category#9, i_class#8, lochierarchy#15, rank_within_parent#48] -Arguments: 100, [lochierarchy#15 DESC NULLS LAST, CASE WHEN (lochierarchy#15 = 0) THEN i_category#9 END ASC NULLS FIRST, rank_within_parent#48 ASC NULLS FIRST], [total_sum#12, i_category#9, i_class#8, lochierarchy#15, rank_within_parent#48] +Input [5]: [total_sum#25, i_category#9, i_class#8, lochierarchy#28, rank_within_parent#41] +Arguments: 100, [lochierarchy#28 DESC NULLS LAST, CASE WHEN (lochierarchy#28 = 0) THEN i_category#9 END ASC NULLS FIRST, rank_within_parent#41 ASC NULLS FIRST], [total_sum#25, i_category#9, i_class#8, lochierarchy#28, rank_within_parent#41] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/simplified.txt index 5adee59be..255e4acc3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/simplified.txt @@ -1,64 +1,48 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (12) + WholeStageCodegen (2) Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (11) - Sort [lochierarchy,_w0,total_sum] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [lochierarchy,_w0] #1 - 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 (9) - HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - InputAdapter - Union - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_category,i_class,sum,ws_net_paid] - CometProject [ws_net_paid,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometFilter [i_item_sk,i_class,i_category] - 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] #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 (8) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty] - InputAdapter - 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 - ReusedExchange [i_category,i_class,sum] #3 + CometSort [total_sum,i_category,i_class,lochierarchy,_w0] + CometColumnarExchange [lochierarchy,_w0] #1 + CometHashAggregate [total_sum,i_category,i_class,lochierarchy,_w0,g_category,g_class] + CometColumnarExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 + CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] + CometUnion [total_sum,i_category,i_class,g_category,g_class,lochierarchy] + CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,sum(UnscaledValue(ws_net_paid))] + CometColumnarExchange [i_category,i_class] #3 + CometHashAggregate [i_category,i_class,sum,ws_net_paid] + CometProject [ws_net_paid,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] + CometProject [ws_item_sk,ws_net_paid] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometFilter [i_item_sk,i_class,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty,sum(total_sum)] + CometColumnarExchange [i_category] #7 + CometHashAggregate [i_category,sum,isEmpty,total_sum] + CometHashAggregate [total_sum,i_category,i_class,sum,sum(UnscaledValue(ws_net_paid))] + ReusedExchange [i_category,i_class,sum] #3 + CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty,sum(total_sum)] + CometColumnarExchange #8 + CometHashAggregate [sum,isEmpty,total_sum] + CometHashAggregate [total_sum,i_category,i_class,sum,sum(UnscaledValue(ws_net_paid))] + ReusedExchange [i_category,i_class,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/explain.txt index 799687a19..8b0facb13 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/explain.txt @@ -1,27 +1,29 @@ == Physical Plan == -* 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) +* ColumnarToRow (25) ++- CometSort (24) + +- CometColumnarExchange (23) + +- RowToColumnar (22) + +- * Project (21) + +- Window (20) + +- * ColumnarToRow (19) + +- CometSort (18) + +- CometColumnarExchange (17) + +- CometHashAggregate (16) + +- CometColumnarExchange (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 @@ -93,73 +95,77 @@ Input [6]: [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, 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))] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(16) CometHashAggregate 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] -(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=2] +(17) CometColumnarExchange +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(18) CometSort +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] +Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15], [i_class#9 ASC NULLS FIRST] -(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 +(19) ColumnarToRow [codegen id : 1] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] (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] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] +Arguments: [sum(_w0#15) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#16], [i_class#9] + +(21) Project [codegen id : 2] +Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, ((_w0#15 * 100) / _we0#16) AS revenueratio#17] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, _we0#16] + +(22) RowToColumnar +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#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#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] +(23) CometColumnarExchange +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] +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#17 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(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=3] +(24) CometSort +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] +Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17], [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#17 ASC NULLS FIRST] -(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 +(25) ColumnarToRow [codegen id : 3] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] ===== 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 (30) ++- * ColumnarToRow (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan parquet spark_catalog.default.date_dim (26) -(24) Scan parquet spark_catalog.default.date_dim +(26) 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 -(25) CometFilter +(27) 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)) -(26) CometProject +(28) CometProject Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(27) ColumnarToRow [codegen id : 1] +(29) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(28) BroadcastExchange +(30) BroadcastExchange Input [1]: [d_date_sk#11] 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-spark3_5/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/simplified.txt index 64b162608..cbc817472 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/simplified.txt @@ -1,41 +1,39 @@ -WholeStageCodegen (5) - Sort [i_category,i_class,i_item_id,i_item_desc,revenueratio] +WholeStageCodegen (3) + ColumnarToRow InputAdapter - Exchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - 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 (3) - Sort [i_class] - InputAdapter - Exchange [i_class] #2 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ss_ext_sales_price] - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_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 #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 + RowToColumnar + WholeStageCodegen (2) + Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometColumnarExchange [i_class] #2 + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ss_ext_sales_price] + CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_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 #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + 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/q10a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/explain.txt index 43d59eb84..f2de73f90 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,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (39) -+- * HashAggregate (38) - +- Exchange (37) - +- * ColumnarToRow (36) +* ColumnarToRow (39) ++- CometTakeOrderedAndProject (38) + +- CometHashAggregate (37) + +- CometColumnarExchange (36) +- CometHashAggregate (35) +- CometProject (34) +- CometBroadcastHashJoin (33) @@ -207,23 +207,21 @@ Input [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purch 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)] -(36) ColumnarToRow [codegen id : 1] +(36) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(37) CometHashAggregate 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)#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] -(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] +(38) CometTakeOrderedAndProject +Input [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#32, cd_purchase_estimate#26, cnt2#33, cd_credit_rating#27, cnt3#34, cd_dep_count#28, cnt4#35, cd_dep_employed_count#29, cnt5#36, cd_dep_college_count#30, cnt6#37] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[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], output=[cd_gender#23,cd_marital_status#24,cd_education_status#25,cnt1#32,cd_purchase_estimate#26,cnt2#33,cd_credit_rating#27,cnt3#34,cd_dep_count#28,cnt4#35,cd_dep_employed_count#29,cnt5#36,cd_dep_college_count#30,cnt6#37]), [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#32, cd_purchase_estimate#26, cnt2#33, cd_credit_rating#27, cnt3#34, cd_dep_count#28, cnt4#35, cd_dep_employed_count#29, cnt5#36, cd_dep_college_count#30, cnt6#37], 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#32, cd_purchase_estimate#26, cnt2#33, cd_credit_rating#27, cnt3#34, cd_dep_count#28, cnt4#35, cd_dep_employed_count#29, cnt5#36, cd_dep_college_count#30, cnt6#37] + +(39) ColumnarToRow [codegen id : 1] +Input [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#32, cd_purchase_estimate#26, cnt2#33, cd_credit_rating#27, cnt3#34, cd_dep_count#28, cnt4#35, cd_dep_employed_count#29, cnt5#36, cd_dep_college_count#30, cnt6#37] ===== Subqueries ===== 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 22c208f2a..243bdbb91 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,53 +1,51 @@ -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 (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 (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,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,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 [c_current_cdemo_sk] - CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk] - CometProject [c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customer_sk] - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] - CometFilter [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] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [customer_sk] #5 - CometUnion [customer_sk] - CometProject [ws_bill_customer_sk] [customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,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_ship_customer_sk,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 [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - CometBroadcastExchange [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] #7 - CometFilter [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] - 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] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cnt1,cd_purchase_estimate,cnt2,cd_credit_rating,cnt3,cd_dep_count,cnt4,cd_dep_employed_count,cnt5,cd_dep_college_count,cnt6] + CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cnt1,cd_purchase_estimate,cnt2,cd_credit_rating,cnt3,cd_dep_count,cnt4,cd_dep_employed_count,cnt5,cd_dep_college_count,cnt6,count,count(1)] + CometColumnarExchange [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 + 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,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,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 [c_current_cdemo_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk] + CometProject [c_current_cdemo_sk,c_current_addr_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customer_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [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] + CometBroadcastExchange [ss_customer_sk] #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [customer_sk] #5 + CometUnion [customer_sk] + CometProject [ws_bill_customer_sk] [customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,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_ship_customer_sk,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 [ca_address_sk] #6 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_county] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + CometBroadcastExchange [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] #7 + CometFilter [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] + 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 2b4fa79cd..16a25285c 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 @@ -1,75 +1,83 @@ == Physical Plan == -TakeOrderedAndProject (71) -+- * Project (70) - +- * BroadcastHashJoin Inner BuildRight (69) - :- * 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) - +- * 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) +TakeOrderedAndProject (79) ++- * Project (78) + +- * BroadcastHashJoin Inner BuildRight (77) + :- * Project (59) + : +- * BroadcastHashJoin Inner BuildRight (58) + : :- * BroadcastHashJoin Inner BuildRight (39) + : : :- * Filter (19) + : : : +- * HashAggregate (18) + : : : +- * ColumnarToRow (17) + : : : +- CometColumnarExchange (16) + : : : +- RowToColumnar (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 (38) + : : +- * HashAggregate (37) + : : +- * ColumnarToRow (36) + : : +- CometColumnarExchange (35) + : : +- RowToColumnar (34) + : : +- * HashAggregate (33) + : : +- * ColumnarToRow (32) + : : +- CometProject (31) + : : +- CometBroadcastHashJoin (30) + : : :- CometProject (26) + : : : +- CometBroadcastHashJoin (25) + : : : :- CometFilter (21) + : : : : +- CometScan parquet spark_catalog.default.customer (20) + : : : +- CometBroadcastExchange (24) + : : : +- CometFilter (23) + : : : +- CometScan parquet spark_catalog.default.store_sales (22) + : : +- CometBroadcastExchange (29) + : : +- CometFilter (28) + : : +- CometScan parquet spark_catalog.default.date_dim (27) + : +- BroadcastExchange (57) + : +- * Filter (56) + : +- * HashAggregate (55) + : +- * ColumnarToRow (54) + : +- CometColumnarExchange (53) + : +- RowToColumnar (52) + : +- * HashAggregate (51) + : +- * ColumnarToRow (50) + : +- CometProject (49) + : +- CometBroadcastHashJoin (48) + : :- CometProject (46) + : : +- CometBroadcastHashJoin (45) + : : :- CometFilter (41) + : : : +- CometScan parquet spark_catalog.default.customer (40) + : : +- CometBroadcastExchange (44) + : : +- CometFilter (43) + : : +- CometScan parquet spark_catalog.default.web_sales (42) + : +- ReusedExchange (47) + +- BroadcastExchange (76) + +- * HashAggregate (75) + +- * ColumnarToRow (74) + +- CometColumnarExchange (73) + +- RowToColumnar (72) + +- * HashAggregate (71) + +- * ColumnarToRow (70) + +- CometProject (69) + +- CometBroadcastHashJoin (68) + :- CometProject (66) + : +- CometBroadcastHashJoin (65) + : :- CometFilter (61) + : : +- CometScan parquet spark_catalog.default.customer (60) + : +- CometBroadcastExchange (64) + : +- CometFilter (63) + : +- CometScan parquet spark_catalog.default.web_sales (62) + +- ReusedExchange (67) (1) Scan parquet spark_catalog.default.customer @@ -142,33 +150,39 @@ Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discoun 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] -(15) Exchange +(15) RowToColumnar 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=1] -(16) HashAggregate [codegen id : 8] +(16) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(17) ColumnarToRow [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] + +(18) 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] -(17) Filter [codegen id : 8] +(19) Filter [codegen id : 8] Input [2]: [customer_id#19, year_total#20] Condition : (isnotnull(year_total#20) AND (year_total#20 > 0.00)) -(18) Scan parquet spark_catalog.default.customer +(20) 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 -(19) CometFilter +(21) 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)) -(20) Scan parquet spark_catalog.default.store_sales +(22) 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 Location: InMemoryFileIndex [] @@ -176,90 +190,96 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#32), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(21) CometFilter +(23) 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) -(22) CometBroadcastExchange +(24) CometBroadcastExchange Input [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] Arguments: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] -(23) CometBroadcastHashJoin +(25) CometBroadcastHashJoin Left 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] Right output [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] Arguments: [c_customer_sk#21], [ss_customer_sk#29], Inner, BuildRight -(24) CometProject +(26) CometProject 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] 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, ss_sold_date_sk#32], [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] -(25) Scan parquet spark_catalog.default.date_dim +(27) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#34, d_year#35] 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 -(26) CometFilter +(28) 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 +(29) CometBroadcastExchange Input [2]: [d_date_sk#34, d_year#35] Arguments: [d_date_sk#34, d_year#35] -(28) CometBroadcastHashJoin +(30) 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 +(31) 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] +(32) 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] -(31) HashAggregate [codegen id : 2] +(33) 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] -(32) Exchange +(34) RowToColumnar +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] + +(35) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] + +(36) ColumnarToRow [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] -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] -(33) HashAggregate [codegen id : 3] +(37) 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] -(34) BroadcastExchange +(38) 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=3] -(35) BroadcastHashJoin [codegen id : 8] +(39) BroadcastHashJoin [codegen id : 8] Left keys [1]: [customer_id#19] Right keys [1]: [customer_id#38] Join type: Inner Join condition: None -(36) Scan parquet spark_catalog.default.customer +(40) 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 -(37) CometFilter +(41) 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)) -(38) Scan parquet spark_catalog.default.web_sales +(42) 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 [] @@ -267,86 +287,92 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#54), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(39) CometFilter +(43) 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) -(40) CometBroadcastExchange +(44) CometBroadcastExchange Input [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] Arguments: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] -(41) CometBroadcastHashJoin +(45) 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) CometProject +(46) 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: 10] +(47) ReusedExchange [Reuses operator id: 10] Output [2]: [d_date_sk#56, d_year#57] -(44) CometBroadcastHashJoin +(48) 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) CometProject +(49) 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) ColumnarToRow [codegen id : 4] +(50) 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] +(51) 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] -(48) Exchange +(52) RowToColumnar +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] + +(53) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=4] + +(54) ColumnarToRow [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] -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] -(49) HashAggregate [codegen id : 5] +(55) 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] -(50) Filter [codegen id : 5] +(56) Filter [codegen id : 5] Input [2]: [customer_id#61, year_total#62] Condition : (isnotnull(year_total#62) AND (year_total#62 > 0.00)) -(51) BroadcastExchange +(57) BroadcastExchange Input [2]: [customer_id#61, year_total#62] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(52) BroadcastHashJoin [codegen id : 8] +(58) BroadcastHashJoin [codegen id : 8] Left keys [1]: [customer_id#19] Right keys [1]: [customer_id#61] Join type: Inner Join condition: None -(53) Project [codegen id : 8] +(59) 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] -(54) Scan parquet spark_catalog.default.customer +(60) 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 -(55) CometFilter +(61) 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)) -(56) Scan parquet spark_catalog.default.web_sales +(62) 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 Location: InMemoryFileIndex [] @@ -354,128 +380,134 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#74), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(57) CometFilter +(63) 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) -(58) CometBroadcastExchange +(64) CometBroadcastExchange Input [4]: [ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] Arguments: [ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] -(59) CometBroadcastHashJoin +(65) CometBroadcastHashJoin Left 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] Right output [4]: [ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] Arguments: [c_customer_sk#63], [ws_bill_customer_sk#71], Inner, BuildRight -(60) CometProject +(66) CometProject 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] Arguments: [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], [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] -(61) ReusedExchange [Reuses operator id: 27] +(67) ReusedExchange [Reuses operator id: 29] Output [2]: [d_date_sk#76, d_year#77] -(62) CometBroadcastHashJoin +(68) CometBroadcastHashJoin Left 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] Right output [2]: [d_date_sk#76, d_year#77] Arguments: [ws_sold_date_sk#74], [d_date_sk#76], Inner, BuildRight -(63) CometProject +(69) CometProject 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] Arguments: [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], [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] -(64) ColumnarToRow [codegen id : 6] +(70) ColumnarToRow [codegen id : 6] 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] -(65) HashAggregate [codegen id : 6] +(71) HashAggregate [codegen id : 6] 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] -(66) Exchange +(72) RowToColumnar +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] + +(73) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=6] + +(74) ColumnarToRow [codegen id : 7] 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=6] -(67) HashAggregate [codegen id : 7] +(75) HashAggregate [codegen id : 7] 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] -(68) BroadcastExchange +(76) BroadcastExchange Input [2]: [customer_id#80, year_total#81] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] -(69) BroadcastHashJoin [codegen id : 8] +(77) BroadcastHashJoin [codegen id : 8] Left keys [1]: [customer_id#19] Right keys [1]: [customer_id#80] 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) -(70) Project [codegen id : 8] +(78) 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] -(71) TakeOrderedAndProject +(79) 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] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (75) -+- * ColumnarToRow (74) - +- CometFilter (73) - +- CometScan parquet spark_catalog.default.date_dim (72) +BroadcastExchange (83) ++- * ColumnarToRow (82) + +- CometFilter (81) + +- CometScan parquet spark_catalog.default.date_dim (80) -(72) Scan parquet spark_catalog.default.date_dim +(80) 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 -(73) CometFilter +(81) 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)) -(74) ColumnarToRow [codegen id : 1] +(82) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#14, d_year#15] -(75) BroadcastExchange +(83) BroadcastExchange Input [2]: [d_date_sk#14, d_year#15] 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) -+- * ColumnarToRow (78) - +- CometFilter (77) - +- CometScan parquet spark_catalog.default.date_dim (76) +Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#33 +BroadcastExchange (87) ++- * ColumnarToRow (86) + +- CometFilter (85) + +- CometScan parquet spark_catalog.default.date_dim (84) -(76) Scan parquet spark_catalog.default.date_dim +(84) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#34, d_year#35] 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 +(85) 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)) -(78) ColumnarToRow [codegen id : 1] +(86) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#34, d_year#35] -(79) BroadcastExchange +(87) BroadcastExchange Input [2]: [d_date_sk#34, d_year#35] 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#54 IN dynamicpruning#13 +Subquery:3 Hosting operator id = 42 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 +Subquery:4 Hosting operator id = 62 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 d7c6ef3b2..916aff7c8 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 @@ -7,38 +7,11 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom BroadcastHashJoin [customer_id,customer_id] Filter [year_total] 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 (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] - 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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_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_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - 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 + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 + RowToColumnar + 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] ColumnarToRow InputAdapter @@ -48,59 +21,94 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] 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 [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_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 #8 + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 + CometBroadcastExchange [d_date_sk,d_year] #4 CometFilter [d_date_sk,d_year] 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #6 + RowToColumnar + 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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + 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 [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_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 #8 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter 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] #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] - 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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 - CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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 + ColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 + RowToColumnar + 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] + 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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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 #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] #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] - 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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 - CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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 + ColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 + RowToColumnar + 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] + 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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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 9160db4e6..6144d0adc 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,26 +1,30 @@ == Physical Plan == -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) +TakeOrderedAndProject (26) ++- * Project (25) + +- Window (24) + +- * ColumnarToRow (23) + +- CometSort (22) + +- CometColumnarExchange (21) + +- RowToColumnar (20) + +- * HashAggregate (19) + +- * ColumnarToRow (18) + +- CometColumnarExchange (17) + +- RowToColumnar (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 @@ -97,66 +101,78 @@ Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] 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] -(16) Exchange +(16) RowToColumnar 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] -(17) HashAggregate [codegen id : 2] +(17) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(18) ColumnarToRow [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] + +(19) 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))#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] -(18) Exchange +(20) RowToColumnar +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17] + +(21) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] + +(22) CometSort 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] +Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17], [i_class#9 ASC NULLS FIRST] -(19) Sort [codegen id : 3] +(23) ColumnarToRow [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 -(20) Window +(24) 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] -(21) Project [codegen id : 4] +(25) 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] -(22) TakeOrderedAndProject +(26) 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 (27) -+- * ColumnarToRow (26) - +- CometProject (25) - +- CometFilter (24) - +- CometScan parquet spark_catalog.default.date_dim (23) +BroadcastExchange (31) ++- * ColumnarToRow (30) + +- CometProject (29) + +- CometFilter (28) + +- CometScan parquet spark_catalog.default.date_dim (27) -(23) Scan parquet spark_catalog.default.date_dim +(27) 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 +(28) 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 +(29) CometProject Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(26) ColumnarToRow [codegen id : 1] +(30) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(27) BroadcastExchange +(31) BroadcastExchange Input [1]: [d_date_sk#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-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 07c91b94d..663689cf7 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 @@ -4,35 +4,39 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter Window [_w0,i_class] WholeStageCodegen (3) - Sort [i_class] + ColumnarToRow InputAdapter - Exchange [i_class] #1 - 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 (1) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometColumnarExchange [i_class] #1 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + 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 ca217cde1..2479fafef 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 @@ -1,88 +1,94 @@ == Physical Plan == -TakeOrderedAndProject (84) -+- * BroadcastHashJoin Inner BuildRight (83) - :- * Filter (66) - : +- * HashAggregate (65) - : +- Exchange (64) - : +- * HashAggregate (63) - : +- * Project (62) - : +- * BroadcastHashJoin Inner BuildRight (61) - : :- * Project (59) - : : +- * BroadcastHashJoin Inner BuildRight (58) - : : :- * BroadcastHashJoin LeftSemi BuildRight (51) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (50) - : : : +- * Project (49) - : : : +- * BroadcastHashJoin Inner BuildRight (48) - : : : :- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.item (4) - : : : +- BroadcastExchange (47) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) - : : : :- * HashAggregate (35) - : : : : +- Exchange (34) - : : : : +- * 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) - : : : +- * 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) - : : : +- CometFilter (53) - : : : +- CometScan parquet spark_catalog.default.item (52) - : : +- ReusedExchange (55) - : +- ReusedExchange (60) - +- BroadcastExchange (82) - +- * Filter (81) - +- * HashAggregate (80) - +- Exchange (79) - +- * HashAggregate (78) - +- * Project (77) - +- * BroadcastHashJoin Inner BuildRight (76) - :- * Project (74) - : +- * BroadcastHashJoin Inner BuildRight (73) - : :- * BroadcastHashJoin LeftSemi BuildRight (71) - : : :- * ColumnarToRow (69) - : : : +- CometFilter (68) - : : : +- CometScan parquet spark_catalog.default.store_sales (67) - : : +- ReusedExchange (70) - : +- ReusedExchange (72) - +- ReusedExchange (75) +TakeOrderedAndProject (90) ++- * BroadcastHashJoin Inner BuildRight (89) + :- * Filter (67) + : +- * HashAggregate (66) + : +- * ColumnarToRow (65) + : +- CometColumnarExchange (64) + : +- RowToColumnar (63) + : +- * HashAggregate (62) + : +- * ColumnarToRow (61) + : +- CometProject (60) + : +- CometBroadcastHashJoin (59) + : :- CometProject (54) + : : +- CometBroadcastHashJoin (53) + : : :- CometBroadcastHashJoin (47) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (46) + : : : +- CometProject (45) + : : : +- CometBroadcastHashJoin (44) + : : : :- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.item (3) + : : : +- CometBroadcastExchange (43) + : : : +- CometBroadcastHashJoin (42) + : : : :- CometHashAggregate (32) + : : : : +- CometColumnarExchange (31) + : : : : +- CometHashAggregate (30) + : : : : +- CometProject (29) + : : : : +- CometBroadcastHashJoin (28) + : : : : :- CometProject (26) + : : : : : +- CometBroadcastHashJoin (25) + : : : : : :- CometFilter (6) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (5) + : : : : : +- CometBroadcastExchange (24) + : : : : : +- CometBroadcastHashJoin (23) + : : : : : :- CometFilter (8) + : : : : : : +- CometScan parquet spark_catalog.default.item (7) + : : : : : +- CometBroadcastExchange (22) + : : : : : +- CometProject (21) + : : : : : +- CometBroadcastHashJoin (20) + : : : : : :- CometProject (15) + : : : : : : +- CometBroadcastHashJoin (14) + : : : : : : :- CometFilter (10) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (9) + : : : : : : +- CometBroadcastExchange (13) + : : : : : : +- CometFilter (12) + : : : : : : +- CometScan parquet spark_catalog.default.item (11) + : : : : : +- CometBroadcastExchange (19) + : : : : : +- CometProject (18) + : : : : : +- CometFilter (17) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (16) + : : : : +- ReusedExchange (27) + : : : +- CometBroadcastExchange (41) + : : : +- CometProject (40) + : : : +- CometBroadcastHashJoin (39) + : : : :- CometProject (37) + : : : : +- CometBroadcastHashJoin (36) + : : : : :- CometFilter (34) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (33) + : : : : +- ReusedExchange (35) + : : : +- ReusedExchange (38) + : : +- CometBroadcastExchange (52) + : : +- CometBroadcastHashJoin (51) + : : :- CometFilter (49) + : : : +- CometScan parquet spark_catalog.default.item (48) + : : +- ReusedExchange (50) + : +- CometBroadcastExchange (58) + : +- CometProject (57) + : +- CometFilter (56) + : +- CometScan parquet spark_catalog.default.date_dim (55) + +- BroadcastExchange (88) + +- * Filter (87) + +- * HashAggregate (86) + +- * ColumnarToRow (85) + +- CometColumnarExchange (84) + +- RowToColumnar (83) + +- * HashAggregate (82) + +- * ColumnarToRow (81) + +- CometProject (80) + +- CometBroadcastHashJoin (79) + :- CometProject (74) + : +- CometBroadcastHashJoin (73) + : :- CometBroadcastHashJoin (71) + : : :- CometFilter (69) + : : : +- CometScan parquet spark_catalog.default.store_sales (68) + : : +- ReusedExchange (70) + : +- ReusedExchange (72) + +- CometBroadcastExchange (78) + +- CometProject (77) + +- CometFilter (76) + +- CometScan parquet spark_catalog.default.date_dim (75) (1) Scan parquet spark_catalog.default.store_sales @@ -97,24 +103,18 @@ 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 : 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 +(3) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) -(6) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(7) Scan parquet spark_catalog.default.store_sales +(5) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] @@ -122,22 +122,22 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(8) CometFilter +(6) CometFilter Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(9) Scan parquet spark_catalog.default.item +(7) 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 -(10) CometFilter +(8) 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)) -(11) Scan parquet spark_catalog.default.catalog_sales +(9) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] @@ -145,116 +145,111 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(12) CometFilter +(10) CometFilter Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) -(13) Scan parquet spark_catalog.default.item +(11) 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 -(14) CometFilter +(12) 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) -(15) CometBroadcastExchange +(13) 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] -(16) CometBroadcastHashJoin +(14) 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 -(17) CometProject +(15) 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 +(16) 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 -(19) CometFilter +(17) 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 +(18) CometProject Input [2]: [d_date_sk#24, d_year#25] Arguments: [d_date_sk#24], [d_date_sk#24] -(21) CometBroadcastExchange +(19) CometBroadcastExchange Input [1]: [d_date_sk#24] Arguments: [d_date_sk#24] -(22) CometBroadcastHashJoin +(20) 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 +(21) 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] -(24) CometBroadcastExchange +(22) CometBroadcastExchange Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] -(25) CometBroadcastHashJoin +(23) 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 -(26) CometBroadcastExchange +(24) CometBroadcastExchange Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(27) CometBroadcastHashJoin +(25) 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 -(28) CometProject +(26) 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] -(29) ReusedExchange [Reuses operator id: 21] +(27) ReusedExchange [Reuses operator id: 19] Output [1]: [d_date_sk#26] -(30) CometBroadcastHashJoin +(28) 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 -(31) CometProject +(29) 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] -(32) CometHashAggregate +(30) CometHashAggregate Input [3]: [brand_id#27, class_id#28, category_id#29] Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] -(33) ColumnarToRow [codegen id : 1] +(31) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(34) Exchange -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 : 3] +(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#27, class_id#28, category_id#29] -(36) Scan parquet spark_catalog.default.web_sales +(33) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] @@ -262,488 +257,536 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(37) CometFilter +(34) CometFilter Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] Condition : isnotnull(ws_item_sk#30) -(38) ReusedExchange [Reuses operator id: 15] +(35) ReusedExchange [Reuses operator id: 13] Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -(39) CometBroadcastHashJoin +(36) 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) CometProject +(37) 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) ReusedExchange [Reuses operator id: 21] +(38) ReusedExchange [Reuses operator id: 19] Output [1]: [d_date_sk#37] -(42) CometBroadcastHashJoin +(39) 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) CometProject +(40) 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) ColumnarToRow [codegen id : 2] +(41) CometBroadcastExchange Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] -(45) BroadcastExchange -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 : 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 +(42) CometBroadcastHashJoin +Left output [3]: [brand_id#27, class_id#28, category_id#29] +Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [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)], [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)], LeftSemi, BuildRight -(47) BroadcastExchange +(43) CometBroadcastExchange 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] +Arguments: [brand_id#27, class_id#28, category_id#29] -(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#27, class_id#28, category_id#29] -Join type: Inner -Join condition: None +(44) CometBroadcastHashJoin +Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Right output [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight -(49) Project [codegen id : 4] -Output [1]: [i_item_sk#6 AS ss_item_sk#38] +(45) CometProject 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] +Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] -(50) BroadcastExchange +(46) CometBroadcastExchange Input [1]: [ss_item_sk#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: [ss_item_sk#38] -(51) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#38] -Join type: LeftSemi -Join condition: None +(47) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Right output [1]: [ss_item_sk#38] +Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight -(52) Scan parquet spark_catalog.default.item +(48) Scan parquet spark_catalog.default.item 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 +(49) CometFilter 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 : 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] +(50) ReusedExchange [Reuses operator id: 46] Output [1]: [ss_item_sk#38] -(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 +(51) CometBroadcastHashJoin +Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Right output [1]: [ss_item_sk#38] +Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight -(57) BroadcastExchange +(52) CometBroadcastExchange 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] +Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -(58) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#39] -Join type: Inner -Join condition: None +(53) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight -(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] +(54) CometProject 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] +Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] + +(55) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#43, d_week_seq#44] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#45), IsNotNull(d_date_sk)] +ReadSchema: struct -(60) ReusedExchange [Reuses operator id: 106] -Output [1]: [d_date_sk#43] +(56) CometFilter +Input [2]: [d_date_sk#43, d_week_seq#44] +Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = ReusedSubquery Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) -(61) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#43] -Join type: Inner -Join condition: None +(57) CometProject +Input [2]: [d_date_sk#43, d_week_seq#44] +Arguments: [d_date_sk#43], [d_date_sk#43] -(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] +(58) CometBroadcastExchange +Input [1]: [d_date_sk#43] +Arguments: [d_date_sk#43] + +(59) CometBroadcastHashJoin +Left 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] +Right output [1]: [d_date_sk#43] +Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight + +(60) CometProject 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] +Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] + +(61) ColumnarToRow [codegen id : 1] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -(63) HashAggregate [codegen id : 11] +(62) HashAggregate [codegen id : 1] 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#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] +Aggregate Attributes [3]: [sum#47, isEmpty#48, count#49] +Results [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#50, isEmpty#51, count#52] -(64) Exchange -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] +(63) RowToColumnar +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#50, isEmpty#51, count#52] -(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] +(64) CometColumnarExchange +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#50, isEmpty#51, count#52] +Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(65) ColumnarToRow [codegen id : 4] +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#50, isEmpty#51, count#52] + +(66) HashAggregate [codegen id : 4] +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#50, isEmpty#51, count#52] 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))#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] +Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#53, count(1)#54] +Results [6]: [store AS channel#55, 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))#53 AS sales#56, count(1)#54 AS number_sales#57] -(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) Filter [codegen id : 4] +Input [6]: [channel#55, i_brand_id#40, i_class_id#41, i_category_id#42, sales#56, number_sales#57] +Condition : (isnotnull(sales#56) AND (cast(sales#56 as decimal(32,6)) > cast(Subquery scalar-subquery#58, [id=#59] as decimal(32,6)))) -(67) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60] +(68) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_item_sk#60, ss_quantity#61, ss_list_price#62, ss_sold_date_sk#63] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#60), dynamicpruningexpression(ss_sold_date_sk#60 IN dynamicpruning#61)] +PartitionFilters: [isnotnull(ss_sold_date_sk#63), dynamicpruningexpression(ss_sold_date_sk#63 IN dynamicpruning#64)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(68) CometFilter -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) CometFilter +Input [4]: [ss_item_sk#60, ss_quantity#61, ss_list_price#62, ss_sold_date_sk#63] +Condition : isnotnull(ss_item_sk#60) -(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: 46] +Output [1]: [ss_item_sk#65] -(70) ReusedExchange [Reuses operator id: 50] -Output [1]: [ss_item_sk#62] +(71) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#60, ss_quantity#61, ss_list_price#62, ss_sold_date_sk#63] +Right output [1]: [ss_item_sk#65] +Arguments: [ss_item_sk#60], [ss_item_sk#65], LeftSemi, BuildRight -(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: 52] +Output [4]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69] -(72) ReusedExchange [Reuses operator id: 57] -Output [4]: [i_item_sk#63, i_brand_id#64, i_class_id#65, i_category_id#66] +(73) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#60, ss_quantity#61, ss_list_price#62, ss_sold_date_sk#63] +Right output [4]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69] +Arguments: [ss_item_sk#60], [i_item_sk#66], Inner, BuildRight -(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) CometProject +Input [8]: [ss_item_sk#60, ss_quantity#61, ss_list_price#62, ss_sold_date_sk#63, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69] +Arguments: [ss_quantity#61, ss_list_price#62, ss_sold_date_sk#63, i_brand_id#67, i_class_id#68, i_category_id#69], [ss_quantity#61, ss_list_price#62, ss_sold_date_sk#63, i_brand_id#67, i_class_id#68, i_category_id#69] -(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) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#70, d_week_seq#71] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#72), IsNotNull(d_date_sk)] +ReadSchema: struct -(75) ReusedExchange [Reuses operator id: 120] -Output [1]: [d_date_sk#67] +(76) CometFilter +Input [2]: [d_date_sk#70, d_week_seq#71] +Condition : ((isnotnull(d_week_seq#71) AND (d_week_seq#71 = ReusedSubquery Subquery scalar-subquery#72, [id=#73])) AND isnotnull(d_date_sk#70)) -(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) CometProject +Input [2]: [d_date_sk#70, d_week_seq#71] +Arguments: [d_date_sk#70], [d_date_sk#70] + +(78) CometBroadcastExchange +Input [1]: [d_date_sk#70] +Arguments: [d_date_sk#70] + +(79) CometBroadcastHashJoin +Left output [6]: [ss_quantity#61, ss_list_price#62, ss_sold_date_sk#63, i_brand_id#67, i_class_id#68, i_category_id#69] +Right output [1]: [d_date_sk#70] +Arguments: [ss_sold_date_sk#63], [d_date_sk#70], Inner, BuildRight + +(80) CometProject +Input [7]: [ss_quantity#61, ss_list_price#62, ss_sold_date_sk#63, i_brand_id#67, i_class_id#68, i_category_id#69, d_date_sk#70] +Arguments: [ss_quantity#61, ss_list_price#62, i_brand_id#67, i_class_id#68, i_category_id#69], [ss_quantity#61, ss_list_price#62, i_brand_id#67, i_class_id#68, i_category_id#69] + +(81) ColumnarToRow [codegen id : 2] +Input [5]: [ss_quantity#61, ss_list_price#62, i_brand_id#67, i_class_id#68, i_category_id#69] + +(82) HashAggregate [codegen id : 2] +Input [5]: [ss_quantity#61, ss_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(ss_quantity#61 as decimal(10,0)) * ss_list_price#62)), partial_count(1)] +Aggregate Attributes [3]: [sum#74, isEmpty#75, count#76] +Results [6]: [i_brand_id#67, i_class_id#68, i_category_id#69, sum#77, isEmpty#78, count#79] + +(83) RowToColumnar +Input [6]: [i_brand_id#67, i_class_id#68, i_category_id#69, sum#77, isEmpty#78, count#79] + +(84) CometColumnarExchange +Input [6]: [i_brand_id#67, i_class_id#68, i_category_id#69, sum#77, isEmpty#78, count#79] +Arguments: hashpartitioning(i_brand_id#67, i_class_id#68, i_category_id#69, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(85) ColumnarToRow [codegen id : 3] +Input [6]: [i_brand_id#67, i_class_id#68, i_category_id#69, sum#77, isEmpty#78, count#79] -(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 : 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#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 : 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 : 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#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 : 24] +(86) HashAggregate [codegen id : 3] +Input [6]: [i_brand_id#67, i_class_id#68, i_category_id#69, sum#77, isEmpty#78, count#79] +Keys [3]: [i_brand_id#67, i_class_id#68, i_category_id#69] +Functions [2]: [sum((cast(ss_quantity#61 as decimal(10,0)) * ss_list_price#62)), count(1)] +Aggregate Attributes [2]: [sum((cast(ss_quantity#61 as decimal(10,0)) * ss_list_price#62))#80, count(1)#81] +Results [6]: [store AS channel#82, i_brand_id#67, i_class_id#68, i_category_id#69, sum((cast(ss_quantity#61 as decimal(10,0)) * ss_list_price#62))#80 AS sales#83, count(1)#81 AS number_sales#84] + +(87) Filter [codegen id : 3] +Input [6]: [channel#82, i_brand_id#67, i_class_id#68, i_category_id#69, sales#83, number_sales#84] +Condition : (isnotnull(sales#83) AND (cast(sales#83 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#58, [id=#59] as decimal(32,6)))) + +(88) BroadcastExchange +Input [6]: [channel#82, i_brand_id#67, i_class_id#68, i_category_id#69, sales#83, number_sales#84] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=4] + +(89) BroadcastHashJoin [codegen id : 4] 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] +Right keys [3]: [i_brand_id#67, i_class_id#68, i_category_id#69] Join type: Inner Join condition: None -(84) TakeOrderedAndProject -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] +(90) TakeOrderedAndProject +Input [12]: [channel#55, i_brand_id#40, i_class_id#41, i_category_id#42, sales#56, number_sales#57, channel#82, i_brand_id#67, i_class_id#68, i_category_id#69, sales#83, number_sales#84] +Arguments: 100, [i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#55, i_brand_id#40, i_class_id#41, i_category_id#42, sales#56, number_sales#57, channel#82, i_brand_id#67, i_class_id#68, i_category_id#69, sales#83, number_sales#84] ===== Subqueries ===== -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#79, ss_list_price#80, ss_sold_date_sk#81] +Subquery:1 Hosting operator id = 67 Hosting Expression = Subquery scalar-subquery#58, [id=#59] +* HashAggregate (109) ++- * ColumnarToRow (108) + +- CometColumnarExchange (107) + +- RowToColumnar (106) + +- * HashAggregate (105) + +- * ColumnarToRow (104) + +- CometUnion (103) + :- CometProject (94) + : +- CometBroadcastHashJoin (93) + : :- CometScan parquet spark_catalog.default.store_sales (91) + : +- ReusedExchange (92) + :- CometProject (98) + : +- CometBroadcastHashJoin (97) + : :- CometScan parquet spark_catalog.default.catalog_sales (95) + : +- ReusedExchange (96) + +- CometProject (102) + +- CometBroadcastHashJoin (101) + :- CometScan parquet spark_catalog.default.web_sales (99) + +- ReusedExchange (100) + + +(91) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_quantity#85, ss_list_price#86, ss_sold_date_sk#87] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#81), dynamicpruningexpression(ss_sold_date_sk#81 IN dynamicpruning#82)] +PartitionFilters: [isnotnull(ss_sold_date_sk#87), dynamicpruningexpression(ss_sold_date_sk#87 IN dynamicpruning#88)] ReadSchema: struct -(86) ReusedExchange [Reuses operator id: 21] -Output [1]: [d_date_sk#83] +(92) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#89] -(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 +(93) CometBroadcastHashJoin +Left output [3]: [ss_quantity#85, ss_list_price#86, ss_sold_date_sk#87] +Right output [1]: [d_date_sk#89] +Arguments: [ss_sold_date_sk#87], [d_date_sk#89], Inner, BuildRight -(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] +(94) CometProject +Input [4]: [ss_quantity#85, ss_list_price#86, ss_sold_date_sk#87, d_date_sk#89] +Arguments: [quantity#90, list_price#91], [ss_quantity#85 AS quantity#90, ss_list_price#86 AS list_price#91] -(89) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#86, cs_list_price#87, cs_sold_date_sk#88] +(95) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#92, cs_list_price#93, cs_sold_date_sk#94] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#88), dynamicpruningexpression(cs_sold_date_sk#88 IN dynamicpruning#89)] +PartitionFilters: [isnotnull(cs_sold_date_sk#94), dynamicpruningexpression(cs_sold_date_sk#94 IN dynamicpruning#95)] ReadSchema: struct -(90) ReusedExchange [Reuses operator id: 21] -Output [1]: [d_date_sk#90] +(96) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#96] -(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 +(97) CometBroadcastHashJoin +Left output [3]: [cs_quantity#92, cs_list_price#93, cs_sold_date_sk#94] +Right output [1]: [d_date_sk#96] +Arguments: [cs_sold_date_sk#94], [d_date_sk#96], Inner, BuildRight -(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] +(98) CometProject +Input [4]: [cs_quantity#92, cs_list_price#93, cs_sold_date_sk#94, d_date_sk#96] +Arguments: [quantity#97, list_price#98], [cs_quantity#92 AS quantity#97, cs_list_price#93 AS list_price#98] -(93) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#93, ws_list_price#94, ws_sold_date_sk#95] +(99) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#99, ws_list_price#100, ws_sold_date_sk#101] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#95), dynamicpruningexpression(ws_sold_date_sk#95 IN dynamicpruning#96)] +PartitionFilters: [isnotnull(ws_sold_date_sk#101), dynamicpruningexpression(ws_sold_date_sk#101 IN dynamicpruning#102)] ReadSchema: struct -(94) ReusedExchange [Reuses operator id: 21] -Output [1]: [d_date_sk#97] +(100) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#103] -(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 +(101) CometBroadcastHashJoin +Left output [3]: [ws_quantity#99, ws_list_price#100, ws_sold_date_sk#101] +Right output [1]: [d_date_sk#103] +Arguments: [ws_sold_date_sk#101], [d_date_sk#103], Inner, BuildRight -(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] +(102) CometProject +Input [4]: [ws_quantity#99, ws_list_price#100, ws_sold_date_sk#101, d_date_sk#103] +Arguments: [quantity#104, list_price#105], [ws_quantity#99 AS quantity#104, ws_list_price#100 AS list_price#105] -(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] +(103) CometUnion +Child 0 Input [2]: [quantity#90, list_price#91] +Child 1 Input [2]: [quantity#97, list_price#98] +Child 2 Input [2]: [quantity#104, list_price#105] -(98) ColumnarToRow [codegen id : 1] -Input [2]: [quantity#84, list_price#85] +(104) ColumnarToRow [codegen id : 1] +Input [2]: [quantity#90, list_price#91] -(99) HashAggregate [codegen id : 1] -Input [2]: [quantity#84, list_price#85] +(105) HashAggregate [codegen id : 1] +Input [2]: [quantity#90, list_price#91] Keys: [] -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] +Functions [1]: [partial_avg((cast(quantity#90 as decimal(10,0)) * list_price#91))] +Aggregate Attributes [2]: [sum#106, count#107] +Results [2]: [sum#108, count#109] + +(106) RowToColumnar +Input [2]: [sum#108, count#109] + +(107) CometColumnarExchange +Input [2]: [sum#108, count#109] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(100) Exchange -Input [2]: [sum#102, count#103] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] +(108) ColumnarToRow [codegen id : 2] +Input [2]: [sum#108, count#109] -(101) HashAggregate [codegen id : 2] -Input [2]: [sum#102, count#103] +(109) HashAggregate [codegen id : 2] +Input [2]: [sum#108, count#109] Keys: [] -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] +Functions [1]: [avg((cast(quantity#90 as decimal(10,0)) * list_price#91))] +Aggregate Attributes [1]: [avg((cast(quantity#90 as decimal(10,0)) * list_price#91))#110] +Results [1]: [avg((cast(quantity#90 as decimal(10,0)) * list_price#91))#110 AS average_sales#111] -Subquery:2 Hosting operator id = 85 Hosting Expression = ss_sold_date_sk#81 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 89 Hosting Expression = cs_sold_date_sk#88 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 95 Hosting Expression = cs_sold_date_sk#94 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 93 Hosting Expression = ws_sold_date_sk#95 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 99 Hosting Expression = ws_sold_date_sk#101 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (106) -+- * ColumnarToRow (105) - +- CometProject (104) - +- CometFilter (103) - +- CometScan parquet spark_catalog.default.date_dim (102) +BroadcastExchange (114) ++- * ColumnarToRow (113) + +- CometProject (112) + +- CometFilter (111) + +- CometScan parquet spark_catalog.default.date_dim (110) -(102) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_week_seq#106] +(110) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#43, d_week_seq#44] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#107), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#45), IsNotNull(d_date_sk)] ReadSchema: struct -(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)) +(111) CometFilter +Input [2]: [d_date_sk#43, d_week_seq#44] +Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = ReusedSubquery Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) -(104) CometProject -Input [2]: [d_date_sk#43, d_week_seq#106] +(112) CometProject +Input [2]: [d_date_sk#43, d_week_seq#44] Arguments: [d_date_sk#43], [d_date_sk#43] -(105) ColumnarToRow [codegen id : 1] +(113) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] -(106) BroadcastExchange +(114) BroadcastExchange Input [1]: [d_date_sk#43] -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=6] -Subquery:6 Hosting operator id = 103 Hosting Expression = ReusedSubquery Subquery scalar-subquery#107, [id=#108] +Subquery:6 Hosting operator id = 111 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -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) +Subquery:7 Hosting operator id = 110 Hosting Expression = Subquery scalar-subquery#45, [id=#46] +* ColumnarToRow (118) ++- CometProject (117) + +- CometFilter (116) + +- CometScan parquet spark_catalog.default.date_dim (115) -(107) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#109, d_year#110, d_moy#111, d_dom#112] +(115) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_week_seq#112, d_year#113, d_moy#114, d_dom#115] 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 -(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)) +(116) CometFilter +Input [4]: [d_week_seq#112, d_year#113, d_moy#114, d_dom#115] +Condition : (((((isnotnull(d_year#113) AND isnotnull(d_moy#114)) AND isnotnull(d_dom#115)) AND (d_year#113 = 1999)) AND (d_moy#114 = 12)) AND (d_dom#115 = 16)) -(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] +(117) CometProject +Input [4]: [d_week_seq#112, d_year#113, d_moy#114, d_dom#115] +Arguments: [d_week_seq#112], [d_week_seq#112] -(110) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#109] +(118) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#112] -Subquery:8 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (115) -+- * ColumnarToRow (114) - +- CometProject (113) - +- CometFilter (112) - +- CometScan parquet spark_catalog.default.date_dim (111) +Subquery:8 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 +BroadcastExchange (123) ++- * ColumnarToRow (122) + +- CometProject (121) + +- CometFilter (120) + +- CometScan parquet spark_catalog.default.date_dim (119) -(111) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#113] +(119) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_year#116] 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 -(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)) +(120) CometFilter +Input [2]: [d_date_sk#26, d_year#116] +Condition : (((isnotnull(d_year#116) AND (d_year#116 >= 1998)) AND (d_year#116 <= 2000)) AND isnotnull(d_date_sk#26)) -(113) CometProject -Input [2]: [d_date_sk#26, d_year#113] +(121) CometProject +Input [2]: [d_date_sk#26, d_year#116] Arguments: [d_date_sk#26], [d_date_sk#26] -(114) ColumnarToRow [codegen id : 1] +(122) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] -(115) BroadcastExchange +(123) BroadcastExchange Input [1]: [d_date_sk#26] -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=7] -Subquery:9 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 +Subquery:9 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:10 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 +Subquery:10 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 -Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#55, [id=#56] +Subquery:11 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -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) +Subquery:12 Hosting operator id = 55 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:13 Hosting operator id = 87 Hosting Expression = ReusedSubquery Subquery scalar-subquery#58, [id=#59] -(116) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#67, d_week_seq#114] +Subquery:14 Hosting operator id = 68 Hosting Expression = ss_sold_date_sk#63 IN dynamicpruning#64 +BroadcastExchange (128) ++- * ColumnarToRow (127) + +- CometProject (126) + +- CometFilter (125) + +- CometScan parquet spark_catalog.default.date_dim (124) + + +(124) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#70, d_week_seq#71] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#115), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#72), IsNotNull(d_date_sk)] ReadSchema: struct -(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)) +(125) CometFilter +Input [2]: [d_date_sk#70, d_week_seq#71] +Condition : ((isnotnull(d_week_seq#71) AND (d_week_seq#71 = ReusedSubquery Subquery scalar-subquery#72, [id=#73])) AND isnotnull(d_date_sk#70)) -(118) CometProject -Input [2]: [d_date_sk#67, d_week_seq#114] -Arguments: [d_date_sk#67], [d_date_sk#67] +(126) CometProject +Input [2]: [d_date_sk#70, d_week_seq#71] +Arguments: [d_date_sk#70], [d_date_sk#70] -(119) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#67] +(127) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#70] -(120) BroadcastExchange -Input [1]: [d_date_sk#67] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] +(128) BroadcastExchange +Input [1]: [d_date_sk#70] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:13 Hosting operator id = 117 Hosting Expression = ReusedSubquery Subquery scalar-subquery#115, [id=#116] +Subquery:15 Hosting operator id = 125 Hosting Expression = ReusedSubquery Subquery scalar-subquery#72, [id=#73] -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) +Subquery:16 Hosting operator id = 124 Hosting Expression = Subquery scalar-subquery#72, [id=#73] +* ColumnarToRow (132) ++- CometProject (131) + +- CometFilter (130) + +- CometScan parquet spark_catalog.default.date_dim (129) -(121) Scan parquet spark_catalog.default.date_dim +(129) 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 -(122) CometFilter +(130) 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)) -(123) CometProject +(131) 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] -(124) ColumnarToRow [codegen id : 1] +(132) ColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#117] +Subquery:17 Hosting operator id = 76 Hosting Expression = ReusedSubquery Subquery scalar-subquery#72, [id=#73] + +Subquery:18 Hosting operator id = 75 Hosting Expression = ReusedSubquery Subquery scalar-subquery#72, [id=#73] + 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 cd4817202..1210748c5 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,180 +1,173 @@ 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 (24) + WholeStageCodegen (4) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [sales] Subquery #4 WholeStageCodegen (2) HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - InputAdapter - Exchange #13 + ColumnarToRow + InputAdapter + CometColumnarExchange #14 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [quantity,list_price] [sum,count,sum,count] + ColumnarToRow + InputAdapter + CometUnion [quantity,list_price] + CometProject [ss_quantity,ss_list_price] [quantity,list_price] + CometBroadcastHashJoin [ss_quantity,ss_list_price,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_quantity,cs_list_price,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_quantity,ws_list_price,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] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #1 + RowToColumnar WholeStageCodegen (1) - HashAggregate [quantity,list_price] [sum,count,sum,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] ColumnarToRow InputAdapter - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,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_quantity,cs_list_price,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_quantity,ws_list_price,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 (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] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (4) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - ColumnarToRow - InputAdapter + CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #2 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + Subquery #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometBroadcastExchange [ss_item_sk] #3 + CometProject [i_item_sk] [ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,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] - InputAdapter - BroadcastExchange #4 - 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 (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,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 - CometBroadcastHashJoin [i_item_sk,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 [i_brand_id,i_class_id,i_category_id] #8 - CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [cs_item_sk,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - 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 [d_date_sk] #10 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #10 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ws_item_sk,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - ReusedExchange [d_date_sk] #10 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - BroadcastHashJoin [i_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter + CometBroadcastExchange [brand_id,class_id,category_id] #4 + CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + CometHashAggregate [brand_id,class_id,category_id] + CometColumnarExchange [brand_id,class_id,category_id] #5 + 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,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 + CometBroadcastHashJoin [i_item_sk,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 [i_brand_id,i_class_id,i_category_id] #8 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + 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 [d_date_sk] #10 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #10 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #11 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + ReusedExchange [d_date_sk] #10 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] 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 - ReusedExchange [ss_item_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [ss_item_sk] #3 + CometBroadcastExchange [d_date_sk] #13 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #2 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #2 InputAdapter - BroadcastExchange #14 - WholeStageCodegen (23) + BroadcastExchange #15 + WholeStageCodegen (3) 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] #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] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_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 #16 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #6 - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - InputAdapter - ReusedExchange [ss_item_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - InputAdapter - ReusedExchange [d_date_sk] #16 + ColumnarToRow + InputAdapter + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #16 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + ColumnarToRow + InputAdapter + CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_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 #17 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #6 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + Subquery #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + ReusedExchange [ss_item_sk] #3 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + CometBroadcastExchange [d_date_sk] #18 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #6 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #6 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 b0944131e..44ac98d43 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 @@ -1,129 +1,150 @@ == Physical Plan == -TakeOrderedAndProject (125) -+- * HashAggregate (124) - +- Exchange (123) - +- * HashAggregate (122) - +- Union (121) - :- * HashAggregate (100) - : +- Exchange (99) - : +- * HashAggregate (98) - : +- Union (97) - : :- * Filter (66) - : : +- * HashAggregate (65) - : : +- Exchange (64) - : : +- * HashAggregate (63) - : : +- * Project (62) - : : +- * BroadcastHashJoin Inner BuildRight (61) - : : :- * Project (59) - : : : +- * BroadcastHashJoin Inner BuildRight (58) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (51) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- BroadcastExchange (50) - : : : : +- * Project (49) - : : : : +- * BroadcastHashJoin Inner BuildRight (48) - : : : : :- * ColumnarToRow (6) - : : : : : +- CometFilter (5) - : : : : : +- CometScan parquet spark_catalog.default.item (4) - : : : : +- BroadcastExchange (47) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) - : : : : :- * HashAggregate (35) - : : : : : +- Exchange (34) - : : : : : +- * 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) - : : : : +- * 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) - : : : : +- CometFilter (53) - : : : : +- CometScan parquet spark_catalog.default.item (52) - : : : +- ReusedExchange (55) - : : +- ReusedExchange (60) - : :- * Filter (81) - : : +- * HashAggregate (80) - : : +- Exchange (79) - : : +- * HashAggregate (78) - : : +- * Project (77) - : : +- * BroadcastHashJoin Inner BuildRight (76) - : : :- * Project (74) - : : : +- * BroadcastHashJoin Inner BuildRight (73) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (71) - : : : : :- * ColumnarToRow (69) - : : : : : +- CometFilter (68) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (67) - : : : : +- ReusedExchange (70) - : : : +- ReusedExchange (72) - : : +- ReusedExchange (75) - : +- * Filter (96) - : +- * HashAggregate (95) - : +- Exchange (94) - : +- * HashAggregate (93) - : +- * Project (92) - : +- * BroadcastHashJoin Inner BuildRight (91) - : :- * Project (89) - : : +- * BroadcastHashJoin Inner BuildRight (88) - : : :- * BroadcastHashJoin LeftSemi BuildRight (86) - : : : :- * ColumnarToRow (84) - : : : : +- CometFilter (83) - : : : : +- CometScan parquet spark_catalog.default.web_sales (82) - : : : +- ReusedExchange (85) - : : +- ReusedExchange (87) - : +- ReusedExchange (90) - :- * HashAggregate (105) - : +- Exchange (104) - : +- * HashAggregate (103) - : +- * HashAggregate (102) - : +- ReusedExchange (101) - :- * HashAggregate (110) - : +- Exchange (109) - : +- * HashAggregate (108) - : +- * HashAggregate (107) - : +- ReusedExchange (106) - :- * HashAggregate (115) - : +- Exchange (114) - : +- * HashAggregate (113) - : +- * HashAggregate (112) - : +- ReusedExchange (111) - +- * HashAggregate (120) - +- Exchange (119) - +- * HashAggregate (118) - +- * HashAggregate (117) - +- ReusedExchange (116) +* ColumnarToRow (146) ++- CometTakeOrderedAndProject (145) + +- CometHashAggregate (144) + +- CometColumnarExchange (143) + +- RowToColumnar (142) + +- * HashAggregate (141) + +- Union (140) + :- * HashAggregate (107) + : +- * ColumnarToRow (106) + : +- CometColumnarExchange (105) + : +- RowToColumnar (104) + : +- * HashAggregate (103) + : +- Union (102) + : :- * Filter (67) + : : +- * HashAggregate (66) + : : +- * ColumnarToRow (65) + : : +- CometColumnarExchange (64) + : : +- RowToColumnar (63) + : : +- * HashAggregate (62) + : : +- * ColumnarToRow (61) + : : +- CometProject (60) + : : +- CometBroadcastHashJoin (59) + : : :- CometProject (54) + : : : +- CometBroadcastHashJoin (53) + : : : :- CometBroadcastHashJoin (47) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometBroadcastExchange (46) + : : : : +- CometProject (45) + : : : : +- CometBroadcastHashJoin (44) + : : : : :- CometFilter (4) + : : : : : +- CometScan parquet spark_catalog.default.item (3) + : : : : +- CometBroadcastExchange (43) + : : : : +- CometBroadcastHashJoin (42) + : : : : :- CometHashAggregate (32) + : : : : : +- CometColumnarExchange (31) + : : : : : +- CometHashAggregate (30) + : : : : : +- CometProject (29) + : : : : : +- CometBroadcastHashJoin (28) + : : : : : :- CometProject (26) + : : : : : : +- CometBroadcastHashJoin (25) + : : : : : : :- CometFilter (6) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (5) + : : : : : : +- CometBroadcastExchange (24) + : : : : : : +- CometBroadcastHashJoin (23) + : : : : : : :- CometFilter (8) + : : : : : : : +- CometScan parquet spark_catalog.default.item (7) + : : : : : : +- CometBroadcastExchange (22) + : : : : : : +- CometProject (21) + : : : : : : +- CometBroadcastHashJoin (20) + : : : : : : :- CometProject (15) + : : : : : : : +- CometBroadcastHashJoin (14) + : : : : : : : :- CometFilter (10) + : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (9) + : : : : : : : +- CometBroadcastExchange (13) + : : : : : : : +- CometFilter (12) + : : : : : : : +- CometScan parquet spark_catalog.default.item (11) + : : : : : : +- CometBroadcastExchange (19) + : : : : : : +- CometProject (18) + : : : : : : +- CometFilter (17) + : : : : : : +- CometScan parquet spark_catalog.default.date_dim (16) + : : : : : +- ReusedExchange (27) + : : : : +- CometBroadcastExchange (41) + : : : : +- CometProject (40) + : : : : +- CometBroadcastHashJoin (39) + : : : : :- CometProject (37) + : : : : : +- CometBroadcastHashJoin (36) + : : : : : :- CometFilter (34) + : : : : : : +- CometScan parquet spark_catalog.default.web_sales (33) + : : : : : +- ReusedExchange (35) + : : : : +- ReusedExchange (38) + : : : +- CometBroadcastExchange (52) + : : : +- CometBroadcastHashJoin (51) + : : : :- CometFilter (49) + : : : : +- CometScan parquet spark_catalog.default.item (48) + : : : +- ReusedExchange (50) + : : +- CometBroadcastExchange (58) + : : +- CometProject (57) + : : +- CometFilter (56) + : : +- CometScan parquet spark_catalog.default.date_dim (55) + : :- * Filter (84) + : : +- * HashAggregate (83) + : : +- * ColumnarToRow (82) + : : +- CometColumnarExchange (81) + : : +- RowToColumnar (80) + : : +- * HashAggregate (79) + : : +- * ColumnarToRow (78) + : : +- CometProject (77) + : : +- CometBroadcastHashJoin (76) + : : :- CometProject (74) + : : : +- CometBroadcastHashJoin (73) + : : : :- CometBroadcastHashJoin (71) + : : : : :- CometFilter (69) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (68) + : : : : +- ReusedExchange (70) + : : : +- ReusedExchange (72) + : : +- ReusedExchange (75) + : +- * Filter (101) + : +- * HashAggregate (100) + : +- * ColumnarToRow (99) + : +- CometColumnarExchange (98) + : +- RowToColumnar (97) + : +- * HashAggregate (96) + : +- * ColumnarToRow (95) + : +- CometProject (94) + : +- CometBroadcastHashJoin (93) + : :- CometProject (91) + : : +- CometBroadcastHashJoin (90) + : : :- CometBroadcastHashJoin (88) + : : : :- CometFilter (86) + : : : : +- CometScan parquet spark_catalog.default.web_sales (85) + : : : +- ReusedExchange (87) + : : +- ReusedExchange (89) + : +- ReusedExchange (92) + :- * HashAggregate (115) + : +- * ColumnarToRow (114) + : +- CometColumnarExchange (113) + : +- RowToColumnar (112) + : +- * HashAggregate (111) + : +- * HashAggregate (110) + : +- * ColumnarToRow (109) + : +- ReusedExchange (108) + :- * HashAggregate (123) + : +- * ColumnarToRow (122) + : +- CometColumnarExchange (121) + : +- RowToColumnar (120) + : +- * HashAggregate (119) + : +- * HashAggregate (118) + : +- * ColumnarToRow (117) + : +- ReusedExchange (116) + :- * HashAggregate (131) + : +- * ColumnarToRow (130) + : +- CometColumnarExchange (129) + : +- RowToColumnar (128) + : +- * HashAggregate (127) + : +- * HashAggregate (126) + : +- * ColumnarToRow (125) + : +- ReusedExchange (124) + +- * HashAggregate (139) + +- * ColumnarToRow (138) + +- CometColumnarExchange (137) + +- RowToColumnar (136) + +- * HashAggregate (135) + +- * HashAggregate (134) + +- * ColumnarToRow (133) + +- ReusedExchange (132) (1) Scan parquet spark_catalog.default.store_sales @@ -138,24 +159,18 @@ 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 : 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 +(3) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) -(6) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(7) Scan parquet spark_catalog.default.store_sales +(5) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] @@ -163,22 +178,22 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(8) CometFilter +(6) CometFilter Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(9) Scan parquet spark_catalog.default.item +(7) 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 -(10) CometFilter +(8) 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)) -(11) Scan parquet spark_catalog.default.catalog_sales +(9) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] @@ -186,116 +201,111 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(12) CometFilter +(10) CometFilter Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) -(13) Scan parquet spark_catalog.default.item +(11) 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 -(14) CometFilter +(12) 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) -(15) CometBroadcastExchange +(13) 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] -(16) CometBroadcastHashJoin +(14) 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 -(17) CometProject +(15) 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 +(16) 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 -(19) CometFilter +(17) 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 +(18) CometProject Input [2]: [d_date_sk#24, d_year#25] Arguments: [d_date_sk#24], [d_date_sk#24] -(21) CometBroadcastExchange +(19) CometBroadcastExchange Input [1]: [d_date_sk#24] Arguments: [d_date_sk#24] -(22) CometBroadcastHashJoin +(20) 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 +(21) 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] -(24) CometBroadcastExchange +(22) CometBroadcastExchange Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] -(25) CometBroadcastHashJoin +(23) 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 -(26) CometBroadcastExchange +(24) CometBroadcastExchange Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(27) CometBroadcastHashJoin +(25) 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 -(28) CometProject +(26) 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] -(29) ReusedExchange [Reuses operator id: 21] +(27) ReusedExchange [Reuses operator id: 19] Output [1]: [d_date_sk#26] -(30) CometBroadcastHashJoin +(28) 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 -(31) CometProject +(29) 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] -(32) CometHashAggregate +(30) CometHashAggregate Input [3]: [brand_id#27, class_id#28, category_id#29] Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] -(33) ColumnarToRow [codegen id : 1] -Input [3]: [brand_id#27, class_id#28, category_id#29] - -(34) Exchange +(31) CometColumnarExchange 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] +Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(35) HashAggregate [codegen id : 3] +(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#27, class_id#28, category_id#29] -(36) Scan parquet spark_catalog.default.web_sales +(33) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] @@ -303,671 +313,744 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(37) CometFilter +(34) CometFilter Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] Condition : isnotnull(ws_item_sk#30) -(38) ReusedExchange [Reuses operator id: 15] +(35) ReusedExchange [Reuses operator id: 13] Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -(39) CometBroadcastHashJoin +(36) 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) CometProject +(37) 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) ReusedExchange [Reuses operator id: 21] +(38) ReusedExchange [Reuses operator id: 19] Output [1]: [d_date_sk#37] -(42) CometBroadcastHashJoin +(39) 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) CometProject +(40) 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) ColumnarToRow [codegen id : 2] -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(45) BroadcastExchange +(41) CometBroadcastExchange 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] +Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] -(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 +(42) CometBroadcastHashJoin +Left output [3]: [brand_id#27, class_id#28, category_id#29] +Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [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)], [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)], LeftSemi, BuildRight -(47) BroadcastExchange +(43) CometBroadcastExchange 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] +Arguments: [brand_id#27, class_id#28, category_id#29] -(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#27, class_id#28, category_id#29] -Join type: Inner -Join condition: None +(44) CometBroadcastHashJoin +Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Right output [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight -(49) Project [codegen id : 4] -Output [1]: [i_item_sk#6 AS ss_item_sk#38] +(45) CometProject 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] +Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] -(50) BroadcastExchange +(46) CometBroadcastExchange Input [1]: [ss_item_sk#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: [ss_item_sk#38] -(51) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#38] -Join type: LeftSemi -Join condition: None +(47) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Right output [1]: [ss_item_sk#38] +Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight -(52) Scan parquet spark_catalog.default.item +(48) Scan parquet spark_catalog.default.item 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 +(49) CometFilter 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 : 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] +(50) ReusedExchange [Reuses operator id: 46] Output [1]: [ss_item_sk#38] -(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 +(51) CometBroadcastHashJoin +Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Right output [1]: [ss_item_sk#38] +Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight -(57) BroadcastExchange +(52) CometBroadcastExchange 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] +Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -(58) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#39] -Join type: Inner -Join condition: None +(53) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight -(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] +(54) CometProject 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] +Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -(60) ReusedExchange [Reuses operator id: 155] -Output [1]: [d_date_sk#43] +(55) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#43, d_year#44, d_moy#45] +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 -(61) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#43] -Join type: Inner -Join condition: None +(56) CometFilter +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) -(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] +(57) CometProject +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Arguments: [d_date_sk#43], [d_date_sk#43] + +(58) CometBroadcastExchange +Input [1]: [d_date_sk#43] +Arguments: [d_date_sk#43] + +(59) CometBroadcastHashJoin +Left 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] +Right output [1]: [d_date_sk#43] +Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight + +(60) CometProject 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] +Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] + +(61) ColumnarToRow [codegen id : 1] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -(63) HashAggregate [codegen id : 11] +(62) HashAggregate [codegen id : 1] 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#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] +Aggregate Attributes [3]: [sum#46, isEmpty#47, count#48] +Results [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#49, isEmpty#50, count#51] + +(63) RowToColumnar +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#49, isEmpty#50, count#51] -(64) Exchange -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] +(64) CometColumnarExchange +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#49, isEmpty#50, count#51] +Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(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] +(65) ColumnarToRow [codegen id : 2] +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#49, isEmpty#50, count#51] + +(66) HashAggregate [codegen id : 2] +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#49, isEmpty#50, count#51] 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))#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] +Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#52, count(1)#53] +Results [6]: [store AS channel#54, 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))#52 AS sales#55, count(1)#53 AS number_sales#56] -(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) Filter [codegen id : 2] +Input [6]: [channel#54, i_brand_id#40, i_class_id#41, i_category_id#42, sales#55, number_sales#56] +Condition : (isnotnull(sales#55) AND (cast(sales#55 as decimal(32,6)) > cast(Subquery scalar-subquery#57, [id=#58] as decimal(32,6)))) -(67) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] +(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] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_sold_date_sk#60 IN dynamicpruning#61)] +PartitionFilters: [isnotnull(cs_sold_date_sk#62), dynamicpruningexpression(cs_sold_date_sk#62 IN dynamicpruning#63)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(68) CometFilter -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 : 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#62] - -(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#63, i_brand_id#64, i_class_id#65, i_category_id#66] - -(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 : 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: 155] -Output [1]: [d_date_sk#67] - -(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 : 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 : 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#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 : 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 : 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#79, ws_quantity#80, ws_list_price#81, ws_sold_date_sk#82] +(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) + +(70) ReusedExchange [Reuses operator id: 46] +Output [1]: [ss_item_sk#64] + +(71) CometBroadcastHashJoin +Left output [4]: [cs_item_sk#59, cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62] +Right output [1]: [ss_item_sk#64] +Arguments: [cs_item_sk#59], [ss_item_sk#64], LeftSemi, BuildRight + +(72) ReusedExchange [Reuses operator id: 52] +Output [4]: [i_item_sk#65, i_brand_id#66, i_class_id#67, i_category_id#68] + +(73) CometBroadcastHashJoin +Left output [4]: [cs_item_sk#59, cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62] +Right output [4]: [i_item_sk#65, i_brand_id#66, i_class_id#67, i_category_id#68] +Arguments: [cs_item_sk#59], [i_item_sk#65], Inner, BuildRight + +(74) CometProject +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] +Arguments: [cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62, i_brand_id#66, i_class_id#67, i_category_id#68], [cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62, i_brand_id#66, i_class_id#67, i_category_id#68] + +(75) ReusedExchange [Reuses operator id: 58] +Output [1]: [d_date_sk#69] + +(76) CometBroadcastHashJoin +Left 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] +Right output [1]: [d_date_sk#69] +Arguments: [cs_sold_date_sk#62], [d_date_sk#69], Inner, BuildRight + +(77) CometProject +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] +Arguments: [cs_quantity#60, cs_list_price#61, i_brand_id#66, i_class_id#67, i_category_id#68], [cs_quantity#60, cs_list_price#61, i_brand_id#66, i_class_id#67, i_category_id#68] + +(78) ColumnarToRow [codegen id : 3] +Input [5]: [cs_quantity#60, cs_list_price#61, i_brand_id#66, i_class_id#67, i_category_id#68] + +(79) HashAggregate [codegen id : 3] +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] + +(80) RowToColumnar +Input [6]: [i_brand_id#66, i_class_id#67, i_category_id#68, sum#73, isEmpty#74, count#75] + +(81) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=3] + +(82) ColumnarToRow [codegen id : 4] +Input [6]: [i_brand_id#66, i_class_id#67, i_category_id#68, sum#73, isEmpty#74, count#75] + +(83) HashAggregate [codegen id : 4] +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 [6]: [catalog AS channel#78, 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#79, count(1)#77 AS number_sales#80] + +(84) Filter [codegen id : 4] +Input [6]: [channel#78, i_brand_id#66, i_class_id#67, i_category_id#68, sales#79, number_sales#80] +Condition : (isnotnull(sales#79) AND (cast(sales#79 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#57, [id=#58] as decimal(32,6)))) + +(85) 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] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#82), dynamicpruningexpression(ws_sold_date_sk#82 IN dynamicpruning#83)] +PartitionFilters: [isnotnull(ws_sold_date_sk#84), dynamicpruningexpression(ws_sold_date_sk#84 IN dynamicpruning#85)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(83) CometFilter -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 : 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#84] - -(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#85, i_brand_id#86, i_class_id#87, i_category_id#88] - -(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 : 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: 155] -Output [1]: [d_date_sk#89] - -(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 : 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 : 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#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 : 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 : 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 : 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#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 : 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#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#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 : 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#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#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 : 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#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#156, sum#170, isEmpty#171, sum#172] -Arguments: hashpartitioning(channel#156, 5), ENSURE_REQUIREMENTS, [plan_id=12] - -(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#180, i_brand_id#181, i_class_id#182, i_category_id#183, sum#184, isEmpty#185, sum#186] - -(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 : 193] -Input [2]: [sum_sales#189, number_sales#190] +(86) 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) + +(87) ReusedExchange [Reuses operator id: 46] +Output [1]: [ss_item_sk#86] + +(88) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#81, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#84] +Right output [1]: [ss_item_sk#86] +Arguments: [ws_item_sk#81], [ss_item_sk#86], LeftSemi, BuildRight + +(89) ReusedExchange [Reuses operator id: 52] +Output [4]: [i_item_sk#87, i_brand_id#88, i_class_id#89, i_category_id#90] + +(90) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#81, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#84] +Right output [4]: [i_item_sk#87, i_brand_id#88, i_class_id#89, i_category_id#90] +Arguments: [ws_item_sk#81], [i_item_sk#87], Inner, BuildRight + +(91) CometProject +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] +Arguments: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#84, i_brand_id#88, i_class_id#89, i_category_id#90], [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#84, i_brand_id#88, i_class_id#89, i_category_id#90] + +(92) ReusedExchange [Reuses operator id: 58] +Output [1]: [d_date_sk#91] + +(93) CometBroadcastHashJoin +Left 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] +Right output [1]: [d_date_sk#91] +Arguments: [ws_sold_date_sk#84], [d_date_sk#91], Inner, BuildRight + +(94) CometProject +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] +Arguments: [ws_quantity#82, ws_list_price#83, i_brand_id#88, i_class_id#89, i_category_id#90], [ws_quantity#82, ws_list_price#83, i_brand_id#88, i_class_id#89, i_category_id#90] + +(95) ColumnarToRow [codegen id : 5] +Input [5]: [ws_quantity#82, ws_list_price#83, i_brand_id#88, i_class_id#89, i_category_id#90] + +(96) HashAggregate [codegen id : 5] +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] + +(97) RowToColumnar +Input [6]: [i_brand_id#88, i_class_id#89, i_category_id#90, sum#95, isEmpty#96, count#97] + +(98) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=4] + +(99) ColumnarToRow [codegen id : 6] +Input [6]: [i_brand_id#88, i_class_id#89, i_category_id#90, sum#95, isEmpty#96, count#97] + +(100) HashAggregate [codegen id : 6] +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 [6]: [web AS channel#100, 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#101, count(1)#99 AS number_sales#102] + +(101) Filter [codegen id : 6] +Input [6]: [channel#100, i_brand_id#88, i_class_id#89, i_category_id#90, sales#101, number_sales#102] +Condition : (isnotnull(sales#101) AND (cast(sales#101 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#57, [id=#58] as decimal(32,6)))) + +(102) Union + +(103) HashAggregate [codegen id : 7] +Input [6]: [channel#54, i_brand_id#40, i_class_id#41, i_category_id#42, sales#55, number_sales#56] +Keys [4]: [channel#54, i_brand_id#40, i_class_id#41, i_category_id#42] +Functions [2]: [partial_sum(sales#55), partial_sum(number_sales#56)] +Aggregate Attributes [3]: [sum#103, isEmpty#104, sum#105] +Results [7]: [channel#54, i_brand_id#40, i_class_id#41, i_category_id#42, sum#106, isEmpty#107, sum#108] + +(104) RowToColumnar +Input [7]: [channel#54, i_brand_id#40, i_class_id#41, i_category_id#42, sum#106, isEmpty#107, sum#108] + +(105) CometColumnarExchange +Input [7]: [channel#54, i_brand_id#40, i_class_id#41, i_category_id#42, sum#106, isEmpty#107, sum#108] +Arguments: hashpartitioning(channel#54, i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(106) ColumnarToRow [codegen id : 8] +Input [7]: [channel#54, i_brand_id#40, i_class_id#41, i_category_id#42, sum#106, isEmpty#107, sum#108] + +(107) HashAggregate [codegen id : 8] +Input [7]: [channel#54, i_brand_id#40, i_class_id#41, i_category_id#42, sum#106, isEmpty#107, sum#108] +Keys [4]: [channel#54, i_brand_id#40, i_class_id#41, i_category_id#42] +Functions [2]: [sum(sales#55), sum(number_sales#56)] +Aggregate Attributes [2]: [sum(sales#55)#109, sum(number_sales#56)#110] +Results [6]: [channel#54, i_brand_id#40, i_class_id#41, i_category_id#42, sum(sales#55)#109 AS sum_sales#111, sum(number_sales#56)#110 AS number_sales#112] + +(108) ReusedExchange [Reuses operator id: 105] +Output [7]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, sum#117, isEmpty#118, sum#119] + +(109) ColumnarToRow [codegen id : 16] +Input [7]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, sum#117, isEmpty#118, sum#119] + +(110) HashAggregate [codegen id : 16] +Input [7]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, sum#117, isEmpty#118, sum#119] +Keys [4]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116] +Functions [2]: [sum(sales#120), sum(number_sales#121)] +Aggregate Attributes [2]: [sum(sales#120)#109, sum(number_sales#121)#110] +Results [5]: [channel#113, i_brand_id#114, i_class_id#115, sum(sales#120)#109 AS sum_sales#122, sum(number_sales#121)#110 AS number_sales#123] + +(111) HashAggregate [codegen id : 16] +Input [5]: [channel#113, i_brand_id#114, i_class_id#115, sum_sales#122, number_sales#123] +Keys [3]: [channel#113, i_brand_id#114, i_class_id#115] +Functions [2]: [partial_sum(sum_sales#122), partial_sum(number_sales#123)] +Aggregate Attributes [3]: [sum#124, isEmpty#125, sum#126] +Results [6]: [channel#113, i_brand_id#114, i_class_id#115, sum#127, isEmpty#128, sum#129] + +(112) RowToColumnar +Input [6]: [channel#113, i_brand_id#114, i_class_id#115, sum#127, isEmpty#128, sum#129] + +(113) CometColumnarExchange +Input [6]: [channel#113, i_brand_id#114, i_class_id#115, sum#127, isEmpty#128, sum#129] +Arguments: hashpartitioning(channel#113, i_brand_id#114, i_class_id#115, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(114) ColumnarToRow [codegen id : 17] +Input [6]: [channel#113, i_brand_id#114, i_class_id#115, sum#127, isEmpty#128, sum#129] + +(115) HashAggregate [codegen id : 17] +Input [6]: [channel#113, i_brand_id#114, i_class_id#115, sum#127, isEmpty#128, sum#129] +Keys [3]: [channel#113, i_brand_id#114, i_class_id#115] +Functions [2]: [sum(sum_sales#122), sum(number_sales#123)] +Aggregate Attributes [2]: [sum(sum_sales#122)#130, sum(number_sales#123)#131] +Results [6]: [channel#113, i_brand_id#114, i_class_id#115, null AS i_category_id#132, sum(sum_sales#122)#130 AS sum(sum_sales)#133, sum(number_sales#123)#131 AS sum(number_sales)#134] + +(116) ReusedExchange [Reuses operator id: 105] +Output [7]: [channel#135, i_brand_id#136, i_class_id#137, i_category_id#138, sum#139, isEmpty#140, sum#141] + +(117) ColumnarToRow [codegen id : 25] +Input [7]: [channel#135, i_brand_id#136, i_class_id#137, i_category_id#138, sum#139, isEmpty#140, sum#141] + +(118) HashAggregate [codegen id : 25] +Input [7]: [channel#135, i_brand_id#136, i_class_id#137, i_category_id#138, sum#139, isEmpty#140, sum#141] +Keys [4]: [channel#135, i_brand_id#136, i_class_id#137, i_category_id#138] +Functions [2]: [sum(sales#142), sum(number_sales#143)] +Aggregate Attributes [2]: [sum(sales#142)#109, sum(number_sales#143)#110] +Results [4]: [channel#135, i_brand_id#136, sum(sales#142)#109 AS sum_sales#144, sum(number_sales#143)#110 AS number_sales#145] + +(119) HashAggregate [codegen id : 25] +Input [4]: [channel#135, i_brand_id#136, sum_sales#144, number_sales#145] +Keys [2]: [channel#135, i_brand_id#136] +Functions [2]: [partial_sum(sum_sales#144), partial_sum(number_sales#145)] +Aggregate Attributes [3]: [sum#146, isEmpty#147, sum#148] +Results [5]: [channel#135, i_brand_id#136, sum#149, isEmpty#150, sum#151] + +(120) RowToColumnar +Input [5]: [channel#135, i_brand_id#136, sum#149, isEmpty#150, sum#151] + +(121) CometColumnarExchange +Input [5]: [channel#135, i_brand_id#136, sum#149, isEmpty#150, sum#151] +Arguments: hashpartitioning(channel#135, i_brand_id#136, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(122) ColumnarToRow [codegen id : 26] +Input [5]: [channel#135, i_brand_id#136, sum#149, isEmpty#150, sum#151] + +(123) HashAggregate [codegen id : 26] +Input [5]: [channel#135, i_brand_id#136, sum#149, isEmpty#150, sum#151] +Keys [2]: [channel#135, i_brand_id#136] +Functions [2]: [sum(sum_sales#144), sum(number_sales#145)] +Aggregate Attributes [2]: [sum(sum_sales#144)#152, sum(number_sales#145)#153] +Results [6]: [channel#135, i_brand_id#136, null AS i_class_id#154, null AS i_category_id#155, sum(sum_sales#144)#152 AS sum(sum_sales)#156, sum(number_sales#145)#153 AS sum(number_sales)#157] + +(124) ReusedExchange [Reuses operator id: 105] +Output [7]: [channel#158, i_brand_id#159, i_class_id#160, i_category_id#161, sum#162, isEmpty#163, sum#164] + +(125) ColumnarToRow [codegen id : 34] +Input [7]: [channel#158, i_brand_id#159, i_class_id#160, i_category_id#161, sum#162, isEmpty#163, sum#164] + +(126) HashAggregate [codegen id : 34] +Input [7]: [channel#158, i_brand_id#159, i_class_id#160, i_category_id#161, sum#162, isEmpty#163, sum#164] +Keys [4]: [channel#158, i_brand_id#159, i_class_id#160, i_category_id#161] +Functions [2]: [sum(sales#165), sum(number_sales#166)] +Aggregate Attributes [2]: [sum(sales#165)#109, sum(number_sales#166)#110] +Results [3]: [channel#158, sum(sales#165)#109 AS sum_sales#167, sum(number_sales#166)#110 AS number_sales#168] + +(127) HashAggregate [codegen id : 34] +Input [3]: [channel#158, sum_sales#167, number_sales#168] +Keys [1]: [channel#158] +Functions [2]: [partial_sum(sum_sales#167), partial_sum(number_sales#168)] +Aggregate Attributes [3]: [sum#169, isEmpty#170, sum#171] +Results [4]: [channel#158, sum#172, isEmpty#173, sum#174] + +(128) RowToColumnar +Input [4]: [channel#158, sum#172, isEmpty#173, sum#174] + +(129) CometColumnarExchange +Input [4]: [channel#158, sum#172, isEmpty#173, sum#174] +Arguments: hashpartitioning(channel#158, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(130) ColumnarToRow [codegen id : 35] +Input [4]: [channel#158, sum#172, isEmpty#173, sum#174] + +(131) HashAggregate [codegen id : 35] +Input [4]: [channel#158, sum#172, isEmpty#173, sum#174] +Keys [1]: [channel#158] +Functions [2]: [sum(sum_sales#167), sum(number_sales#168)] +Aggregate Attributes [2]: [sum(sum_sales#167)#175, sum(number_sales#168)#176] +Results [6]: [channel#158, null AS i_brand_id#177, null AS i_class_id#178, null AS i_category_id#179, sum(sum_sales#167)#175 AS sum(sum_sales)#180, sum(number_sales#168)#176 AS sum(number_sales)#181] + +(132) ReusedExchange [Reuses operator id: 105] +Output [7]: [channel#182, i_brand_id#183, i_class_id#184, i_category_id#185, sum#186, isEmpty#187, sum#188] + +(133) ColumnarToRow [codegen id : 43] +Input [7]: [channel#182, i_brand_id#183, i_class_id#184, i_category_id#185, sum#186, isEmpty#187, sum#188] + +(134) HashAggregate [codegen id : 43] +Input [7]: [channel#182, i_brand_id#183, i_class_id#184, i_category_id#185, sum#186, isEmpty#187, sum#188] +Keys [4]: [channel#182, i_brand_id#183, i_class_id#184, i_category_id#185] +Functions [2]: [sum(sales#189), sum(number_sales#190)] +Aggregate Attributes [2]: [sum(sales#189)#109, sum(number_sales#190)#110] +Results [2]: [sum(sales#189)#109 AS sum_sales#191, sum(number_sales#190)#110 AS number_sales#192] + +(135) HashAggregate [codegen id : 43] +Input [2]: [sum_sales#191, number_sales#192] Keys: [] -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] +Functions [2]: [partial_sum(sum_sales#191), partial_sum(number_sales#192)] +Aggregate Attributes [3]: [sum#193, isEmpty#194, sum#195] +Results [3]: [sum#196, isEmpty#197, sum#198] + +(136) RowToColumnar +Input [3]: [sum#196, isEmpty#197, sum#198] -(119) Exchange -Input [3]: [sum#194, isEmpty#195, sum#196] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] +(137) CometColumnarExchange +Input [3]: [sum#196, isEmpty#197, sum#198] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(120) HashAggregate [codegen id : 194] -Input [3]: [sum#194, isEmpty#195, sum#196] +(138) ColumnarToRow [codegen id : 44] +Input [3]: [sum#196, isEmpty#197, sum#198] + +(139) HashAggregate [codegen id : 44] +Input [3]: [sum#196, isEmpty#197, sum#198] Keys: [] -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] +Functions [2]: [sum(sum_sales#191), sum(number_sales#192)] +Aggregate Attributes [2]: [sum(sum_sales#191)#199, sum(number_sales#192)#200] +Results [6]: [null AS channel#201, null AS i_brand_id#202, null AS i_class_id#203, null AS i_category_id#204, sum(sum_sales#191)#199 AS sum(sum_sales)#205, sum(number_sales#192)#200 AS sum(number_sales)#206] -(121) Union +(140) Union -(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] +(141) HashAggregate [codegen id : 45] +Input [6]: [channel#54, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#111, number_sales#112] +Keys [6]: [channel#54, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#111, number_sales#112] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#109, number_sales#110] +Results [6]: [channel#54, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#111, number_sales#112] + +(142) RowToColumnar +Input [6]: [channel#54, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#111, number_sales#112] -(123) Exchange -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] +(143) CometColumnarExchange +Input [6]: [channel#54, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#111, number_sales#112] +Arguments: hashpartitioning(channel#54, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#111, number_sales#112, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(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] +(144) CometHashAggregate +Input [6]: [channel#54, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#111, number_sales#112] +Keys [6]: [channel#54, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#111, number_sales#112] Functions: [] -Aggregate Attributes: [] -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#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] +(145) CometTakeOrderedAndProject +Input [6]: [channel#54, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#111, number_sales#112] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#54 ASC NULLS FIRST,i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#54,i_brand_id#40,i_class_id#41,i_category_id#42,sum_sales#111,number_sales#112]), [channel#54, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#111, number_sales#112], 100, [channel#54 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#54, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#111, number_sales#112] + +(146) ColumnarToRow [codegen id : 46] +Input [6]: [channel#54, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#111, number_sales#112] ===== Subqueries ===== -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#205, ss_list_price#206, ss_sold_date_sk#207] +Subquery:1 Hosting operator id = 67 Hosting Expression = Subquery scalar-subquery#57, [id=#58] +* HashAggregate (168) ++- * ColumnarToRow (167) + +- CometColumnarExchange (166) + +- RowToColumnar (165) + +- * HashAggregate (164) + +- * ColumnarToRow (163) + +- CometUnion (162) + :- CometProject (150) + : +- CometBroadcastHashJoin (149) + : :- CometScan parquet spark_catalog.default.store_sales (147) + : +- ReusedExchange (148) + :- CometProject (157) + : +- CometBroadcastHashJoin (156) + : :- CometScan parquet spark_catalog.default.catalog_sales (151) + : +- CometBroadcastExchange (155) + : +- CometProject (154) + : +- CometFilter (153) + : +- CometScan parquet spark_catalog.default.date_dim (152) + +- CometProject (161) + +- CometBroadcastHashJoin (160) + :- CometScan parquet spark_catalog.default.web_sales (158) + +- ReusedExchange (159) + + +(147) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_quantity#207, ss_list_price#208, ss_sold_date_sk#209] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#207), dynamicpruningexpression(ss_sold_date_sk#207 IN dynamicpruning#208)] +PartitionFilters: [isnotnull(ss_sold_date_sk#209), dynamicpruningexpression(ss_sold_date_sk#209 IN dynamicpruning#210)] ReadSchema: struct -(127) ReusedExchange [Reuses operator id: 21] -Output [1]: [d_date_sk#209] +(148) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#211] -(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 +(149) CometBroadcastHashJoin +Left output [3]: [ss_quantity#207, ss_list_price#208, ss_sold_date_sk#209] +Right output [1]: [d_date_sk#211] +Arguments: [ss_sold_date_sk#209], [d_date_sk#211], Inner, BuildRight -(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] +(150) CometProject +Input [4]: [ss_quantity#207, ss_list_price#208, ss_sold_date_sk#209, d_date_sk#211] +Arguments: [quantity#212, list_price#213], [ss_quantity#207 AS quantity#212, ss_list_price#208 AS list_price#213] -(130) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#212, cs_list_price#213, cs_sold_date_sk#214] +(151) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#214, cs_list_price#215, cs_sold_date_sk#216] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#214), dynamicpruningexpression(cs_sold_date_sk#214 IN dynamicpruning#215)] +PartitionFilters: [isnotnull(cs_sold_date_sk#216), dynamicpruningexpression(cs_sold_date_sk#216 IN dynamicpruning#217)] ReadSchema: struct -(131) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#216, d_year#217] +(152) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#218, d_year#219] 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)) +(153) CometFilter +Input [2]: [d_date_sk#218, d_year#219] +Condition : (((isnotnull(d_year#219) AND (d_year#219 >= 1998)) AND (d_year#219 <= 2000)) AND isnotnull(d_date_sk#218)) -(133) CometProject -Input [2]: [d_date_sk#216, d_year#217] -Arguments: [d_date_sk#216], [d_date_sk#216] +(154) CometProject +Input [2]: [d_date_sk#218, d_year#219] +Arguments: [d_date_sk#218], [d_date_sk#218] -(134) CometBroadcastExchange -Input [1]: [d_date_sk#216] -Arguments: [d_date_sk#216] +(155) CometBroadcastExchange +Input [1]: [d_date_sk#218] +Arguments: [d_date_sk#218] -(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 +(156) CometBroadcastHashJoin +Left output [3]: [cs_quantity#214, cs_list_price#215, cs_sold_date_sk#216] +Right output [1]: [d_date_sk#218] +Arguments: [cs_sold_date_sk#216], [d_date_sk#218], Inner, BuildRight -(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] +(157) CometProject +Input [4]: [cs_quantity#214, cs_list_price#215, cs_sold_date_sk#216, d_date_sk#218] +Arguments: [quantity#220, list_price#221], [cs_quantity#214 AS quantity#220, cs_list_price#215 AS list_price#221] -(137) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#220, ws_list_price#221, ws_sold_date_sk#222] +(158) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#222, ws_list_price#223, ws_sold_date_sk#224] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#222), dynamicpruningexpression(ws_sold_date_sk#222 IN dynamicpruning#223)] +PartitionFilters: [isnotnull(ws_sold_date_sk#224), dynamicpruningexpression(ws_sold_date_sk#224 IN dynamicpruning#225)] ReadSchema: struct -(138) ReusedExchange [Reuses operator id: 134] -Output [1]: [d_date_sk#224] +(159) ReusedExchange [Reuses operator id: 155] +Output [1]: [d_date_sk#226] -(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 +(160) CometBroadcastHashJoin +Left output [3]: [ws_quantity#222, ws_list_price#223, ws_sold_date_sk#224] +Right output [1]: [d_date_sk#226] +Arguments: [ws_sold_date_sk#224], [d_date_sk#226], Inner, BuildRight -(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] +(161) CometProject +Input [4]: [ws_quantity#222, ws_list_price#223, ws_sold_date_sk#224, d_date_sk#226] +Arguments: [quantity#227, list_price#228], [ws_quantity#222 AS quantity#227, ws_list_price#223 AS list_price#228] -(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] +(162) CometUnion +Child 0 Input [2]: [quantity#212, list_price#213] +Child 1 Input [2]: [quantity#220, list_price#221] +Child 2 Input [2]: [quantity#227, list_price#228] -(142) ColumnarToRow [codegen id : 1] -Input [2]: [quantity#210, list_price#211] +(163) ColumnarToRow [codegen id : 1] +Input [2]: [quantity#212, list_price#213] -(143) HashAggregate [codegen id : 1] -Input [2]: [quantity#210, list_price#211] +(164) HashAggregate [codegen id : 1] +Input [2]: [quantity#212, list_price#213] Keys: [] -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] +Functions [1]: [partial_avg((cast(quantity#212 as decimal(10,0)) * list_price#213))] +Aggregate Attributes [2]: [sum#229, count#230] +Results [2]: [sum#231, count#232] + +(165) RowToColumnar +Input [2]: [sum#231, count#232] + +(166) CometColumnarExchange +Input [2]: [sum#231, count#232] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(144) Exchange -Input [2]: [sum#229, count#230] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=15] +(167) ColumnarToRow [codegen id : 2] +Input [2]: [sum#231, count#232] -(145) HashAggregate [codegen id : 2] -Input [2]: [sum#229, count#230] +(168) HashAggregate [codegen id : 2] +Input [2]: [sum#231, count#232] Keys: [] -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] +Functions [1]: [avg((cast(quantity#212 as decimal(10,0)) * list_price#213))] +Aggregate Attributes [1]: [avg((cast(quantity#212 as decimal(10,0)) * list_price#213))#233] +Results [1]: [avg((cast(quantity#212 as decimal(10,0)) * list_price#213))#233 AS average_sales#234] -Subquery:2 Hosting operator id = 126 Hosting Expression = ss_sold_date_sk#207 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 147 Hosting Expression = ss_sold_date_sk#209 IN dynamicpruning#12 -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) +Subquery:3 Hosting operator id = 151 Hosting Expression = cs_sold_date_sk#216 IN dynamicpruning#217 +BroadcastExchange (173) ++- * ColumnarToRow (172) + +- CometProject (171) + +- CometFilter (170) + +- CometScan parquet spark_catalog.default.date_dim (169) -(146) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#216, d_year#217] +(169) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#218, d_year#219] 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 -(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)) +(170) CometFilter +Input [2]: [d_date_sk#218, d_year#219] +Condition : (((isnotnull(d_year#219) AND (d_year#219 >= 1998)) AND (d_year#219 <= 2000)) AND isnotnull(d_date_sk#218)) -(148) CometProject -Input [2]: [d_date_sk#216, d_year#217] -Arguments: [d_date_sk#216], [d_date_sk#216] +(171) CometProject +Input [2]: [d_date_sk#218, d_year#219] +Arguments: [d_date_sk#218], [d_date_sk#218] -(149) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#216] +(172) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#218] -(150) BroadcastExchange -Input [1]: [d_date_sk#216] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] +(173) BroadcastExchange +Input [1]: [d_date_sk#218] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -Subquery:4 Hosting operator id = 137 Hosting Expression = ws_sold_date_sk#222 IN dynamicpruning#215 +Subquery:4 Hosting operator id = 158 Hosting Expression = ws_sold_date_sk#224 IN dynamicpruning#217 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (155) -+- * ColumnarToRow (154) - +- CometProject (153) - +- CometFilter (152) - +- CometScan parquet spark_catalog.default.date_dim (151) +BroadcastExchange (178) ++- * ColumnarToRow (177) + +- CometProject (176) + +- CometFilter (175) + +- CometScan parquet spark_catalog.default.date_dim (174) -(151) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#43, d_year#233, d_moy#234] +(174) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#43, d_year#44, d_moy#45] 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 -(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)) +(175) CometFilter +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) -(153) CometProject -Input [3]: [d_date_sk#43, d_year#233, d_moy#234] +(176) CometProject +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] Arguments: [d_date_sk#43], [d_date_sk#43] -(154) ColumnarToRow [codegen id : 1] +(177) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] -(155) BroadcastExchange +(178) BroadcastExchange Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (160) -+- * ColumnarToRow (159) - +- CometProject (158) - +- CometFilter (157) - +- CometScan parquet spark_catalog.default.date_dim (156) +Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 +BroadcastExchange (183) ++- * ColumnarToRow (182) + +- CometProject (181) + +- CometFilter (180) + +- CometScan parquet spark_catalog.default.date_dim (179) -(156) Scan parquet spark_catalog.default.date_dim +(179) 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 -(157) CometFilter +(180) 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)) -(158) CometProject +(181) CometProject Input [2]: [d_date_sk#26, d_year#235] Arguments: [d_date_sk#26], [d_date_sk#26] -(159) ColumnarToRow [codegen id : 1] +(182) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] -(160) BroadcastExchange +(183) BroadcastExchange Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] -Subquery:7 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 +Subquery:7 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#55, [id=#56] +Subquery:9 Hosting operator id = 84 Hosting Expression = ReusedSubquery Subquery scalar-subquery#57, [id=#58] -Subquery:10 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 68 Hosting Expression = cs_sold_date_sk#62 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#55, [id=#56] +Subquery:11 Hosting operator id = 101 Hosting Expression = ReusedSubquery Subquery scalar-subquery#57, [id=#58] -Subquery:12 Hosting operator id = 82 Hosting Expression = ws_sold_date_sk#82 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 85 Hosting Expression = ws_sold_date_sk#84 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 3dbd7bc33..db1e21cb6 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,98 +1,96 @@ -TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - 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 (195) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - InputAdapter - Union - 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 (37) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - InputAdapter - Union - WholeStageCodegen (12) - Filter [sales] - Subquery #3 - WholeStageCodegen (2) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - InputAdapter - Exchange #15 - WholeStageCodegen (1) - HashAggregate [quantity,list_price] [sum,count,sum,count] +WholeStageCodegen (46) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] + CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 + RowToColumnar + WholeStageCodegen (45) + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] + InputAdapter + Union + WholeStageCodegen (8) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id] #2 + RowToColumnar + WholeStageCodegen (7) + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + InputAdapter + Union + WholeStageCodegen (2) + Filter [sales] + Subquery #3 + WholeStageCodegen (2) + HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] ColumnarToRow InputAdapter - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,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_quantity,cs_list_price,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_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #17 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_quantity,ws_list_price,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 (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] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + CometColumnarExchange #16 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [quantity,list_price] [sum,count,sum,count] + ColumnarToRow + InputAdapter + CometUnion [quantity,list_price] + CometProject [ss_quantity,ss_list_price] [quantity,list_price] + CometBroadcastHashJoin [ss_quantity,ss_list_price,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_quantity,cs_list_price,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 #17 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #18 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,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 #6 - 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 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [ws_quantity,ws_list_price] [quantity,list_price] + CometBroadcastHashJoin [ws_quantity,ws_list_price,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] #18 + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #3 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + ColumnarToRow + InputAdapter + CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 WholeStageCodegen (1) ColumnarToRow InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk] #5 + CometProject [i_item_sk] [ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,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 [brand_id,class_id,category_id] #6 + CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + CometHashAggregate [brand_id,class_id,category_id] + CometColumnarExchange [brand_id,class_id,category_id] #7 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,i_brand_id,i_class_id,i_category_id,d_date_sk] @@ -128,11 +126,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #12 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #13 CometProject [i_brand_id,i_class_id,i_category_id] CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] @@ -142,99 +136,106 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 ReusedExchange [d_date_sk] #12 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (9) - BroadcastHashJoin [i_item_sk,ss_item_sk] - 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 - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [d_date_sk] #4 - 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] #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] - Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - BroadcastHashJoin [cs_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 - InputAdapter - ReusedExchange [d_date_sk] #4 - 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] #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] - Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - BroadcastHashJoin [ws_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 - InputAdapter - ReusedExchange [d_date_sk] #4 - 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] #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 (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] #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 (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] #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 (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 #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 - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] + 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] + ReusedExchange [ss_item_sk] #5 + CometBroadcastExchange [d_date_sk] #15 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + WholeStageCodegen (4) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #19 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] + ColumnarToRow + InputAdapter + CometProject [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,ss_item_sk] + CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [ss_item_sk] #5 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 + ReusedExchange [d_date_sk] #15 + WholeStageCodegen (6) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #20 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] + ColumnarToRow + InputAdapter + CometProject [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,ss_item_sk] + CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [ss_item_sk] #5 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 + ReusedExchange [d_date_sk] #15 + WholeStageCodegen (17) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel,i_brand_id,i_class_id] #21 + RowToColumnar + WholeStageCodegen (16) + 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] + ColumnarToRow + InputAdapter + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + WholeStageCodegen (26) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel,i_brand_id] #22 + RowToColumnar + WholeStageCodegen (25) + 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] + ColumnarToRow + InputAdapter + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + WholeStageCodegen (35) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel] #23 + RowToColumnar + WholeStageCodegen (34) + 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] + ColumnarToRow + InputAdapter + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + WholeStageCodegen (44) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange #24 + RowToColumnar + WholeStageCodegen (43) + 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] + ColumnarToRow + InputAdapter + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 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 36ccc06de..37189e42b 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,151 +1,161 @@ == Physical Plan == -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) +TakeOrderedAndProject (157) ++- Union (156) + :- * HashAggregate (41) + : +- * ColumnarToRow (40) + : +- CometColumnarExchange (39) + : +- RowToColumnar (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 (69) + : +- * ColumnarToRow (68) + : +- CometColumnarExchange (67) + : +- RowToColumnar (66) + : +- * HashAggregate (65) + : +- * ColumnarToRow (64) + : +- CometProject (63) + : +- CometBroadcastHashJoin (62) + : :- CometProject (60) + : : +- CometBroadcastHashJoin (59) + : : :- CometProject (57) + : : : +- CometBroadcastHashJoin (56) + : : : :- CometProject (52) + : : : : +- CometBroadcastHashJoin (51) + : : : : :- CometProject (49) + : : : : : +- CometBroadcastHashJoin (48) + : : : : : :- CometProject (46) + : : : : : : +- CometBroadcastHashJoin (45) + : : : : : : :- CometFilter (43) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (42) + : : : : : : +- ReusedExchange (44) + : : : : : +- ReusedExchange (47) + : : : : +- ReusedExchange (50) + : : : +- CometBroadcastExchange (55) + : : : +- CometFilter (54) + : : : +- CometScan parquet spark_catalog.default.customer_address (53) + : : +- ReusedExchange (58) + : +- ReusedExchange (61) + :- * HashAggregate (98) + : +- * ColumnarToRow (97) + : +- CometColumnarExchange (96) + : +- RowToColumnar (95) + : +- * HashAggregate (94) + : +- * ColumnarToRow (93) + : +- CometProject (92) + : +- CometBroadcastHashJoin (91) + : :- CometProject (89) + : : +- CometBroadcastHashJoin (88) + : : :- CometProject (86) + : : : +- CometBroadcastHashJoin (85) + : : : :- CometProject (80) + : : : : +- CometBroadcastHashJoin (79) + : : : : :- CometProject (77) + : : : : : +- CometBroadcastHashJoin (76) + : : : : : :- CometProject (74) + : : : : : : +- CometBroadcastHashJoin (73) + : : : : : : :- CometFilter (71) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (70) + : : : : : : +- ReusedExchange (72) + : : : : : +- ReusedExchange (75) + : : : : +- ReusedExchange (78) + : : : +- CometBroadcastExchange (84) + : : : +- CometProject (83) + : : : +- CometFilter (82) + : : : +- CometScan parquet spark_catalog.default.customer_address (81) + : : +- ReusedExchange (87) + : +- ReusedExchange (90) + :- * HashAggregate (127) + : +- * ColumnarToRow (126) + : +- CometColumnarExchange (125) + : +- RowToColumnar (124) + : +- * HashAggregate (123) + : +- * ColumnarToRow (122) + : +- CometProject (121) + : +- CometBroadcastHashJoin (120) + : :- CometProject (118) + : : +- CometBroadcastHashJoin (117) + : : :- CometProject (115) + : : : +- CometBroadcastHashJoin (114) + : : : :- CometProject (109) + : : : : +- CometBroadcastHashJoin (108) + : : : : :- CometProject (106) + : : : : : +- CometBroadcastHashJoin (105) + : : : : : :- CometProject (103) + : : : : : : +- CometBroadcastHashJoin (102) + : : : : : : :- CometFilter (100) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (99) + : : : : : : +- ReusedExchange (101) + : : : : : +- ReusedExchange (104) + : : : : +- ReusedExchange (107) + : : : +- CometBroadcastExchange (113) + : : : +- CometProject (112) + : : : +- CometFilter (111) + : : : +- CometScan parquet spark_catalog.default.customer_address (110) + : : +- ReusedExchange (116) + : +- ReusedExchange (119) + +- * HashAggregate (155) + +- * ColumnarToRow (154) + +- CometColumnarExchange (153) + +- RowToColumnar (152) + +- * HashAggregate (151) + +- * ColumnarToRow (150) + +- CometProject (149) + +- CometBroadcastHashJoin (148) + :- CometProject (144) + : +- CometBroadcastHashJoin (143) + : :- CometProject (141) + : : +- CometBroadcastHashJoin (140) + : : :- CometProject (138) + : : : +- CometBroadcastHashJoin (137) + : : : :- CometProject (135) + : : : : +- CometBroadcastHashJoin (134) + : : : : :- CometProject (132) + : : : : : +- CometBroadcastHashJoin (131) + : : : : : :- CometFilter (129) + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (128) + : : : : : +- ReusedExchange (130) + : : : : +- ReusedExchange (133) + : : : +- ReusedExchange (136) + : : +- ReusedExchange (139) + : +- ReusedExchange (142) + +- CometBroadcastExchange (147) + +- CometFilter (146) + +- CometScan parquet spark_catalog.default.item (145) (1) Scan parquet spark_catalog.default.catalog_sales @@ -326,18 +336,24 @@ Functions [7]: [partial_avg(agg1#29), partial_avg(agg2#30), partial_avg(agg3#31) 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 +(38) RowToColumnar 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] +(39) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(40) ColumnarToRow [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] + +(41) 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 +(42) 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 [] @@ -345,116 +361,122 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#86), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(41) CometFilter +(43) 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)) -(42) ReusedExchange [Reuses operator id: 6] +(44) ReusedExchange [Reuses operator id: 6] Output [2]: [cd_demo_sk#88, cd_dep_count#89] -(43) CometBroadcastHashJoin +(45) 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 -(44) CometProject +(46) 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] -(45) ReusedExchange [Reuses operator id: 12] +(47) 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] -(46) CometBroadcastHashJoin +(48) 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 -(47) CometProject +(49) 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] -(48) ReusedExchange [Reuses operator id: 17] +(50) ReusedExchange [Reuses operator id: 17] Output [1]: [cd_demo_sk#94] -(49) CometBroadcastHashJoin +(51) 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 -(50) CometProject +(52) 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] -(51) Scan parquet spark_catalog.default.customer_address +(53) 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 -(52) CometFilter +(54) 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 +(55) 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 +(56) 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 +(57) 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] +(58) ReusedExchange [Reuses operator id: 28] Output [1]: [d_date_sk#98] -(57) CometBroadcastHashJoin +(59) 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 +(60) 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] +(61) ReusedExchange [Reuses operator id: 33] Output [2]: [i_item_sk#99, i_item_id#100] -(60) CometBroadcastHashJoin +(62) 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 +(63) 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] +(64) 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] +(65) 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 +(66) RowToColumnar +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] + +(67) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] + +(68) ColumnarToRow [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] -Arguments: hashpartitioning(i_item_id#100, ca_country#97, ca_state#96, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(65) HashAggregate [codegen id : 4] +(69) 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 +(70) 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 [] @@ -462,120 +484,126 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#159), dynamicpruningexpression(cs_s PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(67) CometFilter +(71) 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)) -(68) ReusedExchange [Reuses operator id: 6] +(72) ReusedExchange [Reuses operator id: 6] Output [2]: [cd_demo_sk#161, cd_dep_count#162] -(69) CometBroadcastHashJoin +(73) 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 -(70) CometProject +(74) 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] -(71) ReusedExchange [Reuses operator id: 12] +(75) 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] -(72) CometBroadcastHashJoin +(76) 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 -(73) CometProject +(77) 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] -(74) ReusedExchange [Reuses operator id: 17] +(78) ReusedExchange [Reuses operator id: 17] Output [1]: [cd_demo_sk#167] -(75) CometBroadcastHashJoin +(79) 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 -(76) CometProject +(80) 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] -(77) Scan parquet spark_catalog.default.customer_address +(81) 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 -(78) CometFilter +(82) 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 +(83) 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 +(84) CometBroadcastExchange Input [2]: [ca_address_sk#168, ca_country#170] Arguments: [ca_address_sk#168, ca_country#170] -(81) CometBroadcastHashJoin +(85) 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 +(86) 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] +(87) ReusedExchange [Reuses operator id: 28] Output [1]: [d_date_sk#171] -(84) CometBroadcastHashJoin +(88) 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 +(89) 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] +(90) ReusedExchange [Reuses operator id: 33] Output [2]: [i_item_sk#172, i_item_id#173] -(87) CometBroadcastHashJoin +(91) 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 +(92) 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] +(93) 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] +(94) 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 +(95) RowToColumnar +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] + +(96) CometColumnarExchange 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] +Arguments: hashpartitioning(i_item_id#173, ca_country#170, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(92) HashAggregate [codegen id : 6] +(97) ColumnarToRow [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] + +(98) 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 +(99) 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 [] @@ -583,120 +611,126 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#233), dynamicpruningexpression(cs_s PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(94) CometFilter +(100) 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)) -(95) ReusedExchange [Reuses operator id: 6] +(101) ReusedExchange [Reuses operator id: 6] Output [2]: [cd_demo_sk#235, cd_dep_count#236] -(96) CometBroadcastHashJoin +(102) 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 -(97) CometProject +(103) 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] -(98) ReusedExchange [Reuses operator id: 12] +(104) 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] -(99) CometBroadcastHashJoin +(105) 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 -(100) CometProject +(106) 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] -(101) ReusedExchange [Reuses operator id: 17] +(107) ReusedExchange [Reuses operator id: 17] Output [1]: [cd_demo_sk#241] -(102) CometBroadcastHashJoin +(108) 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 -(103) CometProject +(109) 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] -(104) Scan parquet spark_catalog.default.customer_address +(110) 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 -(105) CometFilter +(111) 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 +(112) CometProject Input [2]: [ca_address_sk#242, ca_state#243] Arguments: [ca_address_sk#242], [ca_address_sk#242] -(107) CometBroadcastExchange +(113) CometBroadcastExchange Input [1]: [ca_address_sk#242] Arguments: [ca_address_sk#242] -(108) CometBroadcastHashJoin +(114) 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 +(115) 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] +(116) ReusedExchange [Reuses operator id: 28] Output [1]: [d_date_sk#244] -(111) CometBroadcastHashJoin +(117) 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 +(118) 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] +(119) ReusedExchange [Reuses operator id: 33] Output [2]: [i_item_sk#245, i_item_id#246] -(114) CometBroadcastHashJoin +(120) 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 +(121) 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] +(122) 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] +(123) 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 +(124) RowToColumnar +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] + +(125) CometColumnarExchange 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] +Arguments: hashpartitioning(i_item_id#246, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(119) HashAggregate [codegen id : 8] +(126) ColumnarToRow [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] + +(127) 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 +(128) 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 [] @@ -704,159 +738,165 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#307), dynamicpruningexpression(cs_s PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(121) CometFilter +(129) 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)) -(122) ReusedExchange [Reuses operator id: 6] +(130) ReusedExchange [Reuses operator id: 6] Output [2]: [cd_demo_sk#309, cd_dep_count#310] -(123) CometBroadcastHashJoin +(131) 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 -(124) CometProject +(132) 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] -(125) ReusedExchange [Reuses operator id: 12] +(133) 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] -(126) CometBroadcastHashJoin +(134) 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 -(127) CometProject +(135) 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] -(128) ReusedExchange [Reuses operator id: 17] +(136) ReusedExchange [Reuses operator id: 17] Output [1]: [cd_demo_sk#315] -(129) CometBroadcastHashJoin +(137) 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 -(130) CometProject +(138) 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] -(131) ReusedExchange [Reuses operator id: 107] +(139) ReusedExchange [Reuses operator id: 113] Output [1]: [ca_address_sk#316] -(132) CometBroadcastHashJoin +(140) 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 -(133) CometProject +(141) 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] -(134) ReusedExchange [Reuses operator id: 28] +(142) ReusedExchange [Reuses operator id: 28] Output [1]: [d_date_sk#317] -(135) CometBroadcastHashJoin +(143) 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 -(136) CometProject +(144) 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] -(137) Scan parquet spark_catalog.default.item +(145) 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 -(138) CometFilter +(146) CometFilter Input [1]: [i_item_sk#318] Condition : isnotnull(i_item_sk#318) -(139) CometBroadcastExchange +(147) CometBroadcastExchange Input [1]: [i_item_sk#318] Arguments: [i_item_sk#318] -(140) CometBroadcastHashJoin +(148) 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 -(141) CometProject +(149) 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] -(142) ColumnarToRow [codegen id : 9] +(150) ColumnarToRow [codegen id : 9] Input [7]: [agg1#319, agg2#320, agg3#321, agg4#322, agg5#323, agg6#324, agg7#325] -(143) HashAggregate [codegen id : 9] +(151) 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#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] -(144) Exchange +(152) RowToColumnar +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] + +(153) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=5] + +(154) ColumnarToRow [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] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5] -(145) HashAggregate [codegen id : 10] +(155) 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#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] -(146) Union +(156) Union -(147) TakeOrderedAndProject +(157) 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 (152) -+- * ColumnarToRow (151) - +- CometProject (150) - +- CometFilter (149) - +- CometScan parquet spark_catalog.default.date_dim (148) +BroadcastExchange (162) ++- * ColumnarToRow (161) + +- CometProject (160) + +- CometFilter (159) + +- CometScan parquet spark_catalog.default.date_dim (158) -(148) Scan parquet spark_catalog.default.date_dim +(158) 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 -(149) CometFilter +(159) 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)) -(150) CometProject +(160) CometProject Input [2]: [d_date_sk#25, d_year#26] Arguments: [d_date_sk#25], [d_date_sk#25] -(151) ColumnarToRow [codegen id : 1] +(161) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] -(152) BroadcastExchange +(162) BroadcastExchange Input [1]: [d_date_sk#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#86 IN dynamicpruning#10 +Subquery:2 Hosting operator id = 42 Hosting Expression = cs_sold_date_sk#86 IN dynamicpruning#10 -Subquery:3 Hosting operator id = 66 Hosting Expression = cs_sold_date_sk#159 IN dynamicpruning#10 +Subquery:3 Hosting operator id = 70 Hosting Expression = cs_sold_date_sk#159 IN dynamicpruning#10 -Subquery:4 Hosting operator id = 93 Hosting Expression = cs_sold_date_sk#233 IN dynamicpruning#10 +Subquery:4 Hosting operator id = 99 Hosting Expression = cs_sold_date_sk#233 IN dynamicpruning#10 -Subquery:5 Hosting operator id = 120 Hosting Expression = cs_sold_date_sk#307 IN dynamicpruning#10 +Subquery:5 Hosting operator id = 128 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 509d4be06..f904dd231 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 @@ -2,178 +2,188 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag Union 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 (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] - 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,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,i_item_sk,i_item_id] - 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_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,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 [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,ca_address_sk,ca_county,ca_state,ca_country] - 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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - 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_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,cd_demo_sk,cd_dep_count] - CometFilter [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] - 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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange [cd_demo_sk] #5 - CometFilter [cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #8 - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id,ca_country,ca_state,ca_county] #1 + RowToColumnar + 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] + 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,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,i_item_sk,i_item_id] + 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_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,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 [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,ca_address_sk,ca_county,ca_state,ca_country] + 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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + 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_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,cd_demo_sk,cd_dep_count] + CometFilter [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] + 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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 + CometProject [cd_demo_sk,cd_dep_count] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometBroadcastExchange [cd_demo_sk] #5 + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] + CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] + CometBroadcastExchange [d_date_sk] #7 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_item_id] #8 + CometFilter [i_item_sk,i_item_id] + 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] #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] - 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,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country,i_item_sk,i_item_id] - 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_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,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 [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,ca_address_sk,ca_state,ca_country] - 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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - 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_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,cd_demo_sk,cd_dep_count] - CometFilter [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] - 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 - 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 [ca_address_sk,ca_state,ca_country] #10 - CometFilter [ca_address_sk,ca_state,ca_country] - 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 + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id,ca_country,ca_state] #9 + RowToColumnar + 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] + 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,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country,i_item_sk,i_item_id] + 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_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,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 [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,ca_address_sk,ca_state,ca_country] + 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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + 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_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,cd_demo_sk,cd_dep_count] + CometFilter [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] + 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 + 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 [ca_address_sk,ca_state,ca_country] #10 + CometFilter [ca_address_sk,ca_state,ca_country] + 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] #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] - 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,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country,i_item_sk,i_item_id] - 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_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,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 [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,ca_address_sk,ca_country] - 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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - 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_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,cd_demo_sk,cd_dep_count] - CometFilter [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] - 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 - 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 [ca_address_sk,ca_country] #12 - CometProject [ca_address_sk,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] - 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 + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id,ca_country] #11 + RowToColumnar + 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] + 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,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country,i_item_sk,i_item_id] + 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_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,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 [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,ca_address_sk,ca_country] + 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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + 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_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,cd_demo_sk,cd_dep_count] + CometFilter [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] + 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 + 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 [ca_address_sk,ca_country] #12 + CometProject [ca_address_sk,ca_country] + CometFilter [ca_address_sk,ca_state,ca_country] + 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] #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] - 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,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk,i_item_id] - 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_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,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 [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,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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - 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_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,cd_demo_sk,cd_dep_count] - CometFilter [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] - 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 - 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 [ca_address_sk] #14 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - 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 + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #13 + RowToColumnar + 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] + 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,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk,i_item_id] + 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_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,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 [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,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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + 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_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,cd_demo_sk,cd_dep_count] + CometFilter [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] + 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 + 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 [ca_address_sk] #14 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + 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 #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] - 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,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,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_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,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 [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,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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - 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_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,cd_demo_sk,cd_dep_count] - CometFilter [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] - 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 - 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 [i_item_sk] #16 - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk] + ColumnarToRow + InputAdapter + CometColumnarExchange #15 + RowToColumnar + 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] + 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,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,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_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,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 [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,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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + 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_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,cd_demo_sk,cd_dep_count] + CometFilter [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] + 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 + 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 [i_item_sk] #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 6619cc9d5..ebbb95b41 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,26 +1,30 @@ == Physical Plan == -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) +TakeOrderedAndProject (26) ++- * Project (25) + +- Window (24) + +- * ColumnarToRow (23) + +- CometSort (22) + +- CometColumnarExchange (21) + +- RowToColumnar (20) + +- * HashAggregate (19) + +- * ColumnarToRow (18) + +- CometColumnarExchange (17) + +- RowToColumnar (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 @@ -97,66 +101,78 @@ Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] 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] -(16) Exchange +(16) RowToColumnar 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] -(17) HashAggregate [codegen id : 2] +(17) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(18) ColumnarToRow [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] + +(19) 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))#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] -(18) Exchange +(20) RowToColumnar +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17] + +(21) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] + +(22) CometSort 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] +Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17], [i_class#9 ASC NULLS FIRST] -(19) Sort [codegen id : 3] +(23) ColumnarToRow [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 -(20) Window +(24) 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] -(21) Project [codegen id : 4] +(25) 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] -(22) TakeOrderedAndProject +(26) 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 (27) -+- * ColumnarToRow (26) - +- CometProject (25) - +- CometFilter (24) - +- CometScan parquet spark_catalog.default.date_dim (23) +BroadcastExchange (31) ++- * ColumnarToRow (30) + +- CometProject (29) + +- CometFilter (28) + +- CometScan parquet spark_catalog.default.date_dim (27) -(23) Scan parquet spark_catalog.default.date_dim +(27) 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 +(28) 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 +(29) CometProject Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(26) ColumnarToRow [codegen id : 1] +(30) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(27) BroadcastExchange +(31) BroadcastExchange Input [1]: [d_date_sk#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-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 cd6bcfd61..8bb081a5c 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 @@ -4,35 +4,39 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter Window [_w0,i_class] WholeStageCodegen (3) - Sort [i_class] + ColumnarToRow InputAdapter - Exchange [i_class] #1 - 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 (1) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometColumnarExchange [i_class] #1 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + 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 2810779ed..1975ac486 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,28 +1,30 @@ == Physical Plan == -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) +TakeOrderedAndProject (26) ++- * HashAggregate (25) + +- * ColumnarToRow (24) + +- CometColumnarExchange (23) + +- RowToColumnar (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 @@ -124,50 +126,56 @@ Functions [1]: [partial_avg(inv_quantity_on_hand#2)] 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] -(22) Exchange +(22) RowToColumnar 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] -(23) HashAggregate [codegen id : 3] +(23) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] + +(24) ColumnarToRow [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] + +(25) 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)#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] -(24) TakeOrderedAndProject +(26) 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 (29) -+- * ColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan parquet spark_catalog.default.date_dim (25) +BroadcastExchange (31) ++- * ColumnarToRow (30) + +- CometProject (29) + +- CometFilter (28) + +- CometScan parquet spark_catalog.default.date_dim (27) -(25) Scan parquet spark_catalog.default.date_dim +(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 -(26) CometFilter +(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)) -(27) CometProject +(29) CometProject Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(28) ColumnarToRow [codegen id : 1] +(30) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(29) BroadcastExchange +(31) BroadcastExchange Input [1]: [d_date_sk#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-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 0e864ab70..f5b281ac6 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,39 +1,41 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] 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 (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 - ColumnarToRow - InputAdapter - CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] - CometProject [inv_item_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.warehouse + ColumnarToRow + InputAdapter + CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 + RowToColumnar + 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 + ColumnarToRow + InputAdapter + CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] + CometProject [inv_item_sk,inv_quantity_on_hand] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + InputAdapter + 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 ac15878d5..cc3ef34d5 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,50 +1,64 @@ == Physical Plan == -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) +TakeOrderedAndProject (60) ++- Union (59) + :- * HashAggregate (26) + : +- * HashAggregate (25) + : +- * HashAggregate (24) + : +- * ColumnarToRow (23) + : +- CometColumnarExchange (22) + : +- RowToColumnar (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 (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) + : +- * ColumnarToRow (33) + : +- CometColumnarExchange (32) + : +- RowToColumnar (31) + : +- * HashAggregate (30) + : +- * HashAggregate (29) + : +- * ColumnarToRow (28) + : +- ReusedExchange (27) + :- * HashAggregate (42) + : +- * ColumnarToRow (41) + : +- CometColumnarExchange (40) + : +- RowToColumnar (39) + : +- * HashAggregate (38) + : +- * HashAggregate (37) + : +- * ColumnarToRow (36) + : +- ReusedExchange (35) + :- * HashAggregate (50) + : +- * ColumnarToRow (49) + : +- CometColumnarExchange (48) + : +- RowToColumnar (47) + : +- * HashAggregate (46) + : +- * HashAggregate (45) + : +- * ColumnarToRow (44) + : +- ReusedExchange (43) + +- * HashAggregate (58) + +- * ColumnarToRow (57) + +- CometColumnarExchange (56) + +- RowToColumnar (55) + +- * HashAggregate (54) + +- * HashAggregate (53) + +- * ColumnarToRow (52) + +- ReusedExchange (51) (1) Scan parquet spark_catalog.default.inventory @@ -145,178 +159,220 @@ Functions [1]: [partial_avg(inv_quantity_on_hand#3)] 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] -(21) Exchange +(21) RowToColumnar 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] -(22) HashAggregate [codegen id : 2] +(22) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(23) ColumnarToRow [codegen id : 2] +Input [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#16, count#17] + +(24) 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)#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] +(25) 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] +(26) 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] +(27) ReusedExchange [Reuses operator id: 22] Output [6]: [i_product_name#30, i_brand#31, i_class#32, i_category#33, sum#34, count#35] -(26) HashAggregate [codegen id : 4] +(28) ColumnarToRow [codegen id : 4] +Input [6]: [i_product_name#30, i_brand#31, i_class#32, i_category#33, sum#34, count#35] + +(29) 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] +(30) 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 +(31) RowToColumnar 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] +(32) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] + +(33) ColumnarToRow [codegen id : 5] +Input [5]: [i_product_name#30, i_brand#31, i_class#32, sum#40, count#41] + +(34) 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] +(35) ReusedExchange [Reuses operator id: 22] Output [6]: [i_product_name#45, i_brand#46, i_class#47, i_category#48, sum#49, count#50] -(31) HashAggregate [codegen id : 7] +(36) ColumnarToRow [codegen id : 7] +Input [6]: [i_product_name#45, i_brand#46, i_class#47, i_category#48, sum#49, count#50] + +(37) 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] +(38) 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 +(39) RowToColumnar +Input [4]: [i_product_name#45, i_brand#46, sum#55, count#56] + +(40) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=3] + +(41) ColumnarToRow [codegen id : 8] 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] +(42) 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] +(43) ReusedExchange [Reuses operator id: 22] Output [6]: [i_product_name#61, i_brand#62, i_class#63, i_category#64, sum#65, count#66] -(36) HashAggregate [codegen id : 10] +(44) ColumnarToRow [codegen id : 10] +Input [6]: [i_product_name#61, i_brand#62, i_class#63, i_category#64, sum#65, count#66] + +(45) 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] +(46) 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 +(47) RowToColumnar +Input [3]: [i_product_name#61, sum#71, count#72] + +(48) CometColumnarExchange Input [3]: [i_product_name#61, sum#71, count#72] -Arguments: hashpartitioning(i_product_name#61, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: hashpartitioning(i_product_name#61, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(39) HashAggregate [codegen id : 11] +(49) ColumnarToRow [codegen id : 11] +Input [3]: [i_product_name#61, sum#71, count#72] + +(50) 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] +(51) ReusedExchange [Reuses operator id: 22] Output [6]: [i_product_name#78, i_brand#79, i_class#80, i_category#81, sum#82, count#83] -(41) HashAggregate [codegen id : 13] +(52) ColumnarToRow [codegen id : 13] +Input [6]: [i_product_name#78, i_brand#79, i_class#80, i_category#81, sum#82, count#83] + +(53) 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] +(54) HashAggregate [codegen id : 13] Input [1]: [qoh#85] Keys: [] Functions [1]: [partial_avg(qoh#85)] Aggregate Attributes [2]: [sum#86, count#87] Results [2]: [sum#88, count#89] -(43) Exchange +(55) RowToColumnar +Input [2]: [sum#88, count#89] + +(56) CometColumnarExchange +Input [2]: [sum#88, count#89] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(57) ColumnarToRow [codegen id : 14] Input [2]: [sum#88, count#89] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5] -(44) HashAggregate [codegen id : 14] +(58) HashAggregate [codegen id : 14] Input [2]: [sum#88, count#89] Keys: [] 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] -(45) Union +(59) Union -(46) TakeOrderedAndProject +(60) 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 (51) -+- * ColumnarToRow (50) - +- CometProject (49) - +- CometFilter (48) - +- CometScan parquet spark_catalog.default.date_dim (47) +BroadcastExchange (65) ++- * ColumnarToRow (64) + +- CometProject (63) + +- CometFilter (62) + +- CometScan parquet spark_catalog.default.date_dim (61) -(47) Scan parquet spark_catalog.default.date_dim +(61) 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 -(48) CometFilter +(62) 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)) -(49) CometProject +(63) CometProject Input [2]: [d_date_sk#6, d_month_seq#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(50) ColumnarToRow [codegen id : 1] +(64) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(51) BroadcastExchange +(65) BroadcastExchange Input [1]: [d_date_sk#6] 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 fc4778a22..9f82911a1 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 @@ -4,71 +4,85 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] 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 (1) - HashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand] [sum,count,sum,count] - ColumnarToRow - InputAdapter - CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] - CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] - CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange [w_warehouse_sk] #5 - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_product_name,i_brand,i_class,i_category] #1 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand] [sum,count,sum,count] + ColumnarToRow + InputAdapter + CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] + CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] + CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometBroadcastExchange [w_warehouse_sk] #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] #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 + ColumnarToRow + InputAdapter + CometColumnarExchange [i_product_name,i_brand,i_class] #6 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 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] #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 + ColumnarToRow + InputAdapter + CometColumnarExchange [i_product_name,i_brand] #7 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 WholeStageCodegen (11) HashAggregate [i_product_name,sum,count] [avg(qoh),i_brand,i_class,i_category,qoh,sum,count] - InputAdapter - 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 + ColumnarToRow + InputAdapter + CometColumnarExchange [i_product_name] #8 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 WholeStageCodegen (14) HashAggregate [sum,count] [avg(qoh),i_product_name,i_brand,i_class,i_category,qoh,sum,count] - InputAdapter - 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 - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + ColumnarToRow + InputAdapter + CometColumnarExchange #9 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/explain.txt index 0cba9b059..118902e57 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/explain.txt @@ -1,52 +1,54 @@ == Physical Plan == -* Sort (48) -+- Exchange (47) - +- * Filter (46) - +- * HashAggregate (45) - +- Exchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- Exchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Project (14) - : : : : +- * SortMergeJoin Inner (13) - : : : : :- * Sort (6) - : : : : : +- Exchange (5) - : : : : : +- * ColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- * Sort (12) - : : : : +- Exchange (11) - : : : : +- * ColumnarToRow (10) - : : : : +- CometProject (9) - : : : : +- CometFilter (8) - : : : : +- CometScan parquet spark_catalog.default.store_returns (7) - : : : +- BroadcastExchange (19) - : : : +- * ColumnarToRow (18) - : : : +- CometProject (17) - : : : +- CometFilter (16) - : : : +- CometScan parquet spark_catalog.default.store (15) - : : +- BroadcastExchange (25) - : : +- * ColumnarToRow (24) - : : +- CometFilter (23) - : : +- CometScan parquet spark_catalog.default.item (22) - : +- BroadcastExchange (31) - : +- * ColumnarToRow (30) - : +- CometFilter (29) - : +- CometScan parquet spark_catalog.default.customer (28) - +- BroadcastExchange (37) - +- * ColumnarToRow (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.customer_address (34) +* ColumnarToRow (50) ++- CometSort (49) + +- CometColumnarExchange (48) + +- RowToColumnar (47) + +- * Filter (46) + +- * HashAggregate (45) + +- * ColumnarToRow (44) + +- CometColumnarExchange (43) + +- RowToColumnar (42) + +- * HashAggregate (41) + +- * HashAggregate (40) + +- * ColumnarToRow (39) + +- CometColumnarExchange (38) + +- RowToColumnar (37) + +- * HashAggregate (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * ColumnarToRow (29) + : +- CometProject (28) + : +- CometBroadcastHashJoin (27) + : :- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometProject (18) + : : : +- CometBroadcastHashJoin (17) + : : : :- CometProject (12) + : : : : +- CometSortMergeJoin (11) + : : : : :- CometSort (5) + : : : : : +- CometColumnarExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometSort (10) + : : : : +- CometColumnarExchange (9) + : : : : +- CometProject (8) + : : : : +- CometFilter (7) + : : : : +- CometScan parquet spark_catalog.default.store_returns (6) + : : : +- CometBroadcastExchange (16) + : : : +- CometProject (15) + : : : +- CometFilter (14) + : : : +- CometScan parquet spark_catalog.default.store (13) + : : +- CometBroadcastExchange (21) + : : +- CometFilter (20) + : : +- CometScan parquet spark_catalog.default.item (19) + : +- CometBroadcastExchange (26) + : +- CometFilter (25) + : +- CometScan parquet spark_catalog.default.customer (24) + +- BroadcastExchange (33) + +- * ColumnarToRow (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.customer_address (30) (1) Scan parquet spark_catalog.default.store_sales @@ -64,370 +66,384 @@ Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND is Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -(4) ColumnarToRow [codegen id : 1] +(4) CometColumnarExchange Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(5) Exchange +(5) CometSort Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] -(6) Sort [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST], false, 0 - -(7) Scan parquet spark_catalog.default.store_returns +(6) Scan parquet spark_catalog.default.store_returns Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(8) CometFilter +(7) CometFilter Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) -(9) CometProject +(8) CometProject Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] -(10) ColumnarToRow [codegen id : 3] +(9) CometColumnarExchange Input [2]: [sr_item_sk#7, sr_ticket_number#8] +Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(11) Exchange +(10) CometSort Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] -(12) Sort [codegen id : 4] -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST], false, 0 +(11) CometSortMergeJoin +Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Right output [2]: [sr_item_sk#7, sr_ticket_number#8] +Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner -(13) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_ticket_number#4, ss_item_sk#1] -Right keys [2]: [sr_ticket_number#8, sr_item_sk#7] -Join type: Inner -Join condition: None - -(14) Project [codegen id : 9] -Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +(12) CometProject Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -(15) Scan parquet spark_catalog.default.store +(13) Scan parquet spark_catalog.default.store Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(16) CometFilter +(14) CometFilter Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(s_zip#14)) -(17) CometProject +(15) CometProject Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Arguments: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14], [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(18) ColumnarToRow [codegen id : 5] +(16) CometBroadcastExchange Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(19) BroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(17) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +Right output [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight -(20) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#10] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 9] -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] +(18) CometProject Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] -(22) Scan parquet spark_catalog.default.item +(19) Scan parquet spark_catalog.default.item Output [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale ), IsNotNull(i_item_sk)] ReadSchema: struct -(23) CometFilter +(20) CometFilter Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Condition : ((isnotnull(i_color#18) AND (i_color#18 = pale )) AND isnotnull(i_item_sk#15)) -(24) ColumnarToRow [codegen id : 6] +(21) CometBroadcastExchange Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Arguments: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(25) BroadcastExchange -Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(22) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] +Right output [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Arguments: [ss_item_sk#1], [i_item_sk#15], Inner, BuildRight -(26) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#15] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 9] -Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +(23) CometProject Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(28) Scan parquet spark_catalog.default.customer +(24) Scan parquet spark_catalog.default.customer Output [5]: [c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] ReadSchema: struct -(29) CometFilter +(25) CometFilter Input [5]: [c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] Condition : ((isnotnull(c_customer_sk#21) AND isnotnull(c_current_addr_sk#22)) AND isnotnull(c_birth_country#25)) -(30) ColumnarToRow [codegen id : 7] -Input [5]: [c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] - -(31) BroadcastExchange +(26) CometBroadcastExchange Input [5]: [c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +Arguments: [c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] -(32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#21] -Join type: Inner -Join condition: None +(27) CometBroadcastHashJoin +Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Right output [5]: [c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] +Arguments: [ss_customer_sk#2], [c_customer_sk#21], Inner, BuildRight -(33) Project [codegen id : 9] -Output [13]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] +(28) CometProject Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] +Arguments: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25], [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] + +(29) ColumnarToRow [codegen id : 2] +Input [13]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] -(34) Scan parquet spark_catalog.default.customer_address +(30) Scan parquet spark_catalog.default.customer_address Output [4]: [ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct -(35) CometFilter +(31) CometFilter Input [4]: [ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] Condition : ((isnotnull(ca_address_sk#26) AND isnotnull(ca_country#29)) AND isnotnull(ca_zip#28)) -(36) ColumnarToRow [codegen id : 8] +(32) ColumnarToRow [codegen id : 1] Input [4]: [ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] -(37) BroadcastExchange +(33) BroadcastExchange Input [4]: [ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] -Arguments: HashedRelationBroadcastMode(List(input[0, int, false], upper(input[3, string, false]), input[2, string, false]),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(input[0, int, false], upper(input[3, string, false]), input[2, string, false]),false), [plan_id=3] -(38) BroadcastHashJoin [codegen id : 9] +(34) BroadcastHashJoin [codegen id : 2] Left keys [3]: [c_current_addr_sk#22, c_birth_country#25, s_zip#14] Right keys [3]: [ca_address_sk#26, upper(ca_country#29), ca_zip#28] Join type: Inner Join condition: None -(39) Project [codegen id : 9] +(35) Project [codegen id : 2] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, ca_state#27] Input [17]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25, ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] -(40) HashAggregate [codegen id : 9] +(36) HashAggregate [codegen id : 2] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, ca_state#27] Keys [10]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum#30] Results [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#31] -(41) Exchange +(37) RowToColumnar Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#31] -Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(42) HashAggregate [codegen id : 10] +(38) CometColumnarExchange +Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#31] +Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(39) ColumnarToRow [codegen id : 3] +Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#31] + +(40) HashAggregate [codegen id : 3] Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#31] Keys [10]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#32] Results [4]: [c_last_name#24, c_first_name#23, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#32,17,2) AS netpaid#33] -(43) HashAggregate [codegen id : 10] +(41) HashAggregate [codegen id : 3] Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, netpaid#33] Keys [3]: [c_last_name#24, c_first_name#23, s_store_name#11] Functions [1]: [partial_sum(netpaid#33)] Aggregate Attributes [2]: [sum#34, isEmpty#35] Results [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#36, isEmpty#37] -(44) Exchange +(42) RowToColumnar +Input [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#36, isEmpty#37] + +(43) CometColumnarExchange Input [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#36, isEmpty#37] -Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=8] +Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) HashAggregate [codegen id : 11] +(44) ColumnarToRow [codegen id : 4] +Input [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#36, isEmpty#37] + +(45) HashAggregate [codegen id : 4] Input [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#36, isEmpty#37] Keys [3]: [c_last_name#24, c_first_name#23, s_store_name#11] Functions [1]: [sum(netpaid#33)] Aggregate Attributes [1]: [sum(netpaid#33)#38] Results [4]: [c_last_name#24, c_first_name#23, s_store_name#11, sum(netpaid#33)#38 AS paid#39] -(46) Filter [codegen id : 11] +(46) Filter [codegen id : 4] Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] Condition : (isnotnull(paid#39) AND (cast(paid#39 as decimal(33,8)) > cast(Subquery scalar-subquery#40, [id=#41] as decimal(33,8)))) -(47) Exchange +(47) RowToColumnar +Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] + +(48) CometColumnarExchange Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] -Arguments: rangepartitioning(c_last_name#24 ASC NULLS FIRST, c_first_name#23 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Arguments: rangepartitioning(c_last_name#24 ASC NULLS FIRST, c_first_name#23 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(48) Sort [codegen id : 12] +(49) CometSort +Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] +Arguments: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39], [c_last_name#24 ASC NULLS FIRST, c_first_name#23 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST] + +(50) ColumnarToRow [codegen id : 5] Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] -Arguments: [c_last_name#24 ASC NULLS FIRST, c_first_name#23 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#40, [id=#41] -* HashAggregate (75) -+- Exchange (74) - +- * HashAggregate (73) - +- * HashAggregate (72) - +- Exchange (71) - +- * HashAggregate (70) - +- * Project (69) - +- * BroadcastHashJoin Inner BuildRight (68) - :- * Project (66) - : +- * BroadcastHashJoin Inner BuildRight (65) - : :- * Project (63) - : : +- * BroadcastHashJoin Inner BuildRight (62) - : : :- * Project (57) - : : : +- * BroadcastHashJoin Inner BuildRight (56) - : : : :- * Project (54) - : : : : +- * SortMergeJoin Inner (53) - : : : : :- * Sort (50) - : : : : : +- ReusedExchange (49) - : : : : +- * Sort (52) - : : : : +- ReusedExchange (51) - : : : +- ReusedExchange (55) - : : +- BroadcastExchange (61) - : : +- * ColumnarToRow (60) - : : +- CometFilter (59) - : : +- CometScan parquet spark_catalog.default.item (58) - : +- ReusedExchange (64) - +- ReusedExchange (67) - - -(49) ReusedExchange [Reuses operator id: 5] +* HashAggregate (81) ++- * ColumnarToRow (80) + +- CometColumnarExchange (79) + +- RowToColumnar (78) + +- * HashAggregate (77) + +- * HashAggregate (76) + +- * ColumnarToRow (75) + +- CometColumnarExchange (74) + +- RowToColumnar (73) + +- * HashAggregate (72) + +- * Project (71) + +- * BroadcastHashJoin Inner BuildRight (70) + :- * ColumnarToRow (68) + : +- CometProject (67) + : +- CometBroadcastHashJoin (66) + : :- CometProject (64) + : : +- CometBroadcastHashJoin (63) + : : :- CometProject (59) + : : : +- CometBroadcastHashJoin (58) + : : : :- CometProject (56) + : : : : +- CometSortMergeJoin (55) + : : : : :- CometSort (52) + : : : : : +- ReusedExchange (51) + : : : : +- CometSort (54) + : : : : +- ReusedExchange (53) + : : : +- ReusedExchange (57) + : : +- CometBroadcastExchange (62) + : : +- CometFilter (61) + : : +- CometScan parquet spark_catalog.default.item (60) + : +- ReusedExchange (65) + +- ReusedExchange (69) + + +(51) ReusedExchange [Reuses operator id: 4] Output [5]: [ss_item_sk#42, ss_customer_sk#43, ss_store_sk#44, ss_ticket_number#45, ss_net_paid#46] -(50) Sort [codegen id : 2] +(52) CometSort Input [5]: [ss_item_sk#42, ss_customer_sk#43, ss_store_sk#44, ss_ticket_number#45, ss_net_paid#46] -Arguments: [ss_ticket_number#45 ASC NULLS FIRST, ss_item_sk#42 ASC NULLS FIRST], false, 0 +Arguments: [ss_item_sk#42, ss_customer_sk#43, ss_store_sk#44, ss_ticket_number#45, ss_net_paid#46], [ss_ticket_number#45 ASC NULLS FIRST, ss_item_sk#42 ASC NULLS FIRST] -(51) ReusedExchange [Reuses operator id: 11] +(53) ReusedExchange [Reuses operator id: 9] Output [2]: [sr_item_sk#47, sr_ticket_number#48] -(52) Sort [codegen id : 4] +(54) CometSort Input [2]: [sr_item_sk#47, sr_ticket_number#48] -Arguments: [sr_ticket_number#48 ASC NULLS FIRST, sr_item_sk#47 ASC NULLS FIRST], false, 0 +Arguments: [sr_item_sk#47, sr_ticket_number#48], [sr_ticket_number#48 ASC NULLS FIRST, sr_item_sk#47 ASC NULLS FIRST] -(53) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_ticket_number#45, ss_item_sk#42] -Right keys [2]: [sr_ticket_number#48, sr_item_sk#47] -Join type: Inner -Join condition: None +(55) CometSortMergeJoin +Left output [5]: [ss_item_sk#42, ss_customer_sk#43, ss_store_sk#44, ss_ticket_number#45, ss_net_paid#46] +Right output [2]: [sr_item_sk#47, sr_ticket_number#48] +Arguments: [ss_ticket_number#45, ss_item_sk#42], [sr_ticket_number#48, sr_item_sk#47], Inner -(54) Project [codegen id : 9] -Output [4]: [ss_item_sk#42, ss_customer_sk#43, ss_store_sk#44, ss_net_paid#46] +(56) CometProject Input [7]: [ss_item_sk#42, ss_customer_sk#43, ss_store_sk#44, ss_ticket_number#45, ss_net_paid#46, sr_item_sk#47, sr_ticket_number#48] +Arguments: [ss_item_sk#42, ss_customer_sk#43, ss_store_sk#44, ss_net_paid#46], [ss_item_sk#42, ss_customer_sk#43, ss_store_sk#44, ss_net_paid#46] -(55) ReusedExchange [Reuses operator id: 19] +(57) ReusedExchange [Reuses operator id: 16] Output [4]: [s_store_sk#49, s_store_name#50, s_state#51, s_zip#52] -(56) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#44] -Right keys [1]: [s_store_sk#49] -Join type: Inner -Join condition: None +(58) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#42, ss_customer_sk#43, ss_store_sk#44, ss_net_paid#46] +Right output [4]: [s_store_sk#49, s_store_name#50, s_state#51, s_zip#52] +Arguments: [ss_store_sk#44], [s_store_sk#49], Inner, BuildRight -(57) Project [codegen id : 9] -Output [6]: [ss_item_sk#42, ss_customer_sk#43, ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52] +(59) CometProject Input [8]: [ss_item_sk#42, ss_customer_sk#43, ss_store_sk#44, ss_net_paid#46, s_store_sk#49, s_store_name#50, s_state#51, s_zip#52] +Arguments: [ss_item_sk#42, ss_customer_sk#43, ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52], [ss_item_sk#42, ss_customer_sk#43, ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52] -(58) Scan parquet spark_catalog.default.item +(60) Scan parquet spark_catalog.default.item Output [6]: [i_item_sk#53, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(59) CometFilter +(61) CometFilter Input [6]: [i_item_sk#53, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58] Condition : isnotnull(i_item_sk#53) -(60) ColumnarToRow [codegen id : 6] +(62) CometBroadcastExchange Input [6]: [i_item_sk#53, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58] +Arguments: [i_item_sk#53, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58] -(61) BroadcastExchange -Input [6]: [i_item_sk#53, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] +(63) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#42, ss_customer_sk#43, ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52] +Right output [6]: [i_item_sk#53, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58] +Arguments: [ss_item_sk#42], [i_item_sk#53], Inner, BuildRight -(62) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#42] -Right keys [1]: [i_item_sk#53] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 9] -Output [10]: [ss_customer_sk#43, ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58] +(64) CometProject Input [12]: [ss_item_sk#42, ss_customer_sk#43, ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52, i_item_sk#53, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58] +Arguments: [ss_customer_sk#43, ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58], [ss_customer_sk#43, ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58] -(64) ReusedExchange [Reuses operator id: 31] +(65) ReusedExchange [Reuses operator id: 26] Output [5]: [c_customer_sk#59, c_current_addr_sk#60, c_first_name#61, c_last_name#62, c_birth_country#63] -(65) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#43] -Right keys [1]: [c_customer_sk#59] -Join type: Inner -Join condition: None +(66) CometBroadcastHashJoin +Left output [10]: [ss_customer_sk#43, ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58] +Right output [5]: [c_customer_sk#59, c_current_addr_sk#60, c_first_name#61, c_last_name#62, c_birth_country#63] +Arguments: [ss_customer_sk#43], [c_customer_sk#59], Inner, BuildRight -(66) Project [codegen id : 9] -Output [13]: [ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58, c_current_addr_sk#60, c_first_name#61, c_last_name#62, c_birth_country#63] +(67) CometProject Input [15]: [ss_customer_sk#43, ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58, c_customer_sk#59, c_current_addr_sk#60, c_first_name#61, c_last_name#62, c_birth_country#63] +Arguments: [ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58, c_current_addr_sk#60, c_first_name#61, c_last_name#62, c_birth_country#63], [ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58, c_current_addr_sk#60, c_first_name#61, c_last_name#62, c_birth_country#63] -(67) ReusedExchange [Reuses operator id: 37] +(68) ColumnarToRow [codegen id : 2] +Input [13]: [ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58, c_current_addr_sk#60, c_first_name#61, c_last_name#62, c_birth_country#63] + +(69) ReusedExchange [Reuses operator id: 33] Output [4]: [ca_address_sk#64, ca_state#65, ca_zip#66, ca_country#67] -(68) BroadcastHashJoin [codegen id : 9] +(70) BroadcastHashJoin [codegen id : 2] Left keys [3]: [c_current_addr_sk#60, c_birth_country#63, s_zip#52] Right keys [3]: [ca_address_sk#64, upper(ca_country#67), ca_zip#66] Join type: Inner Join condition: None -(69) Project [codegen id : 9] +(71) Project [codegen id : 2] Output [11]: [ss_net_paid#46, s_store_name#50, s_state#51, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58, c_first_name#61, c_last_name#62, ca_state#65] Input [17]: [ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58, c_current_addr_sk#60, c_first_name#61, c_last_name#62, c_birth_country#63, ca_address_sk#64, ca_state#65, ca_zip#66, ca_country#67] -(70) HashAggregate [codegen id : 9] +(72) HashAggregate [codegen id : 2] Input [11]: [ss_net_paid#46, s_store_name#50, s_state#51, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58, c_first_name#61, c_last_name#62, ca_state#65] Keys [10]: [c_last_name#62, c_first_name#61, s_store_name#50, ca_state#65, s_state#51, i_color#56, i_current_price#54, i_manager_id#58, i_units#57, i_size#55] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#46))] Aggregate Attributes [1]: [sum#68] Results [11]: [c_last_name#62, c_first_name#61, s_store_name#50, ca_state#65, s_state#51, i_color#56, i_current_price#54, i_manager_id#58, i_units#57, i_size#55, sum#69] -(71) Exchange +(73) RowToColumnar +Input [11]: [c_last_name#62, c_first_name#61, s_store_name#50, ca_state#65, s_state#51, i_color#56, i_current_price#54, i_manager_id#58, i_units#57, i_size#55, sum#69] + +(74) CometColumnarExchange Input [11]: [c_last_name#62, c_first_name#61, s_store_name#50, ca_state#65, s_state#51, i_color#56, i_current_price#54, i_manager_id#58, i_units#57, i_size#55, sum#69] -Arguments: hashpartitioning(c_last_name#62, c_first_name#61, s_store_name#50, ca_state#65, s_state#51, i_color#56, i_current_price#54, i_manager_id#58, i_units#57, i_size#55, 5), ENSURE_REQUIREMENTS, [plan_id=11] +Arguments: hashpartitioning(c_last_name#62, c_first_name#61, s_store_name#50, ca_state#65, s_state#51, i_color#56, i_current_price#54, i_manager_id#58, i_units#57, i_size#55, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(72) HashAggregate [codegen id : 10] +(75) ColumnarToRow [codegen id : 3] +Input [11]: [c_last_name#62, c_first_name#61, s_store_name#50, ca_state#65, s_state#51, i_color#56, i_current_price#54, i_manager_id#58, i_units#57, i_size#55, sum#69] + +(76) HashAggregate [codegen id : 3] Input [11]: [c_last_name#62, c_first_name#61, s_store_name#50, ca_state#65, s_state#51, i_color#56, i_current_price#54, i_manager_id#58, i_units#57, i_size#55, sum#69] Keys [10]: [c_last_name#62, c_first_name#61, s_store_name#50, ca_state#65, s_state#51, i_color#56, i_current_price#54, i_manager_id#58, i_units#57, i_size#55] Functions [1]: [sum(UnscaledValue(ss_net_paid#46))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#46))#32] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#46))#32,17,2) AS netpaid#70] -(73) HashAggregate [codegen id : 10] +(77) HashAggregate [codegen id : 3] Input [1]: [netpaid#70] Keys: [] Functions [1]: [partial_avg(netpaid#70)] Aggregate Attributes [2]: [sum#71, count#72] Results [2]: [sum#73, count#74] -(74) Exchange +(78) RowToColumnar +Input [2]: [sum#73, count#74] + +(79) CometColumnarExchange +Input [2]: [sum#73, count#74] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(80) ColumnarToRow [codegen id : 4] Input [2]: [sum#73, count#74] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] -(75) HashAggregate [codegen id : 11] +(81) HashAggregate [codegen id : 4] Input [2]: [sum#73, count#74] Keys: [] Functions [1]: [avg(netpaid#70)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/simplified.txt index 5b658f1d7..42c283018 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/simplified.txt @@ -1,122 +1,100 @@ -WholeStageCodegen (12) - Sort [c_last_name,c_first_name,s_store_name] +WholeStageCodegen (5) + ColumnarToRow InputAdapter - Exchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (11) - Filter [paid] - Subquery #1 - WholeStageCodegen (11) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - InputAdapter - Exchange #10 - WholeStageCodegen (10) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - InputAdapter - Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 - WholeStageCodegen (9) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - WholeStageCodegen (2) - Sort [ss_ticket_number,ss_item_sk] - InputAdapter - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #4 - InputAdapter - WholeStageCodegen (4) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - ReusedExchange [sr_item_sk,sr_ticket_number] #5 - InputAdapter - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #6 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - InputAdapter - ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 - InputAdapter - ReusedExchange [ca_address_sk,ca_state,ca_zip,ca_country] #9 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - InputAdapter - Exchange [c_last_name,c_first_name,s_store_name] #2 - WholeStageCodegen (10) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + CometSort [c_last_name,c_first_name,s_store_name,paid] + CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 + RowToColumnar + WholeStageCodegen (4) + Filter [paid] + Subquery #1 + WholeStageCodegen (4) + HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] + ColumnarToRow InputAdapter - Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 - WholeStageCodegen (9) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - WholeStageCodegen (2) - Sort [ss_ticket_number,ss_item_sk] - InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + CometColumnarExchange #10 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [netpaid] [sum,count,sum,count] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] + ColumnarToRow + InputAdapter + CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] + CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] + ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #4 + CometSort [sr_item_sk,sr_ticket_number] + ReusedExchange [sr_item_sk,sr_ticket_number] #5 + ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #6 + CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #12 + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 InputAdapter - WholeStageCodegen (4) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - Exchange [sr_ticket_number,sr_item_sk] #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + ReusedExchange [ca_address_sk,ca_state,ca_zip,ca_country] #9 + HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [c_last_name,c_first_name,s_store_name] #2 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] + ColumnarToRow + InputAdapter + CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] + CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #4 + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + CometSort [sr_item_sk,sr_ticket_number] + CometColumnarExchange [sr_ticket_number,sr_item_sk] #5 + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #6 + CometProject [s_store_sk,s_store_name,s_state,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #7 + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) + BroadcastExchange #9 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] + CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] 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 6c4d257c2..c6190b825 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,75 +1,81 @@ == Physical Plan == -TakeOrderedAndProject (71) -+- Union (70) - :- * HashAggregate (28) - : +- Exchange (27) - : +- * HashAggregate (26) - : +- * 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 (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) +TakeOrderedAndProject (77) ++- Union (76) + :- * HashAggregate (30) + : +- * ColumnarToRow (29) + : +- CometColumnarExchange (28) + : +- RowToColumnar (27) + : +- * HashAggregate (26) + : +- * 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 (53) + : +- * ColumnarToRow (52) + : +- CometColumnarExchange (51) + : +- RowToColumnar (50) + : +- * HashAggregate (49) + : +- * ColumnarToRow (48) + : +- CometProject (47) + : +- CometBroadcastHashJoin (46) + : :- CometProject (44) + : : +- CometBroadcastHashJoin (43) + : : :- CometProject (38) + : : : +- CometBroadcastHashJoin (37) + : : : :- CometProject (35) + : : : : +- CometBroadcastHashJoin (34) + : : : : :- CometFilter (32) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (31) + : : : : +- ReusedExchange (33) + : : : +- ReusedExchange (36) + : : +- CometBroadcastExchange (42) + : : +- CometProject (41) + : : +- CometFilter (40) + : : +- CometScan parquet spark_catalog.default.store (39) + : +- ReusedExchange (45) + +- * HashAggregate (75) + +- * ColumnarToRow (74) + +- CometColumnarExchange (73) + +- RowToColumnar (72) + +- * HashAggregate (71) + +- * ColumnarToRow (70) + +- CometProject (69) + +- CometBroadcastHashJoin (68) + :- CometProject (64) + : +- CometBroadcastHashJoin (63) + : :- CometProject (61) + : : +- CometBroadcastHashJoin (60) + : : :- CometProject (58) + : : : +- CometBroadcastHashJoin (57) + : : : :- CometFilter (55) + : : : : +- CometScan parquet spark_catalog.default.store_sales (54) + : : : +- ReusedExchange (56) + : : +- ReusedExchange (59) + : +- ReusedExchange (62) + +- CometBroadcastExchange (67) + +- CometFilter (66) + +- CometScan parquet spark_catalog.default.item (65) (1) Scan parquet spark_catalog.default.store_sales @@ -198,18 +204,24 @@ Functions [4]: [partial_avg(agg1#20), partial_avg(UnscaledValue(agg2#21)), parti 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 +(27) RowToColumnar 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 : 2] +(28) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(29) ColumnarToRow [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] + +(30) 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 +(31) Scan parquet spark_catalog.default.store_sales 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 [] @@ -217,96 +229,102 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#56), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(30) CometFilter +(32) CometFilter 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) ReusedExchange [Reuses operator id: 6] +(33) ReusedExchange [Reuses operator id: 6] Output [1]: [cd_demo_sk#58] -(32) CometBroadcastHashJoin +(34) 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) CometProject +(35) 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) ReusedExchange [Reuses operator id: 12] +(36) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#59] -(35) CometBroadcastHashJoin +(37) 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) CometProject +(38) 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) Scan parquet spark_catalog.default.store +(39) 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 -(38) CometFilter +(40) 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 +(41) CometProject Input [2]: [s_store_sk#60, s_state#61] Arguments: [s_store_sk#60], [s_store_sk#60] -(40) CometBroadcastExchange +(42) CometBroadcastExchange Input [1]: [s_store_sk#60] Arguments: [s_store_sk#60] -(41) CometBroadcastHashJoin +(43) 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 +(44) 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] +(45) ReusedExchange [Reuses operator id: 22] Output [2]: [i_item_sk#62, i_item_id#63] -(44) CometBroadcastHashJoin +(46) 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 +(47) 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] +(48) ColumnarToRow [codegen id : 3] Input [5]: [i_item_id#63, agg1#64, agg2#65, agg3#66, agg4#67] -(47) HashAggregate [codegen id : 3] +(49) 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 +(50) RowToColumnar +Input [9]: [i_item_id#63, sum#76, count#77, sum#78, count#79, sum#80, count#81, sum#82, count#83] + +(51) CometColumnarExchange 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] +Arguments: hashpartitioning(i_item_id#63, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(49) HashAggregate [codegen id : 4] +(52) ColumnarToRow [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] + +(53) 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 +(54) 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 [] @@ -314,131 +332,137 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#101), dynamicpruningexpression(ss_s PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(51) CometFilter +(55) 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)) -(52) ReusedExchange [Reuses operator id: 6] +(56) ReusedExchange [Reuses operator id: 6] Output [1]: [cd_demo_sk#103] -(53) CometBroadcastHashJoin +(57) 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 -(54) CometProject +(58) 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] -(55) ReusedExchange [Reuses operator id: 12] +(59) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#104] -(56) CometBroadcastHashJoin +(60) 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 -(57) CometProject +(61) 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] -(58) ReusedExchange [Reuses operator id: 40] +(62) ReusedExchange [Reuses operator id: 42] Output [1]: [s_store_sk#105] -(59) CometBroadcastHashJoin +(63) 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 -(60) CometProject +(64) 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] -(61) Scan parquet spark_catalog.default.item +(65) 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 -(62) CometFilter +(66) CometFilter Input [1]: [i_item_sk#106] Condition : isnotnull(i_item_sk#106) -(63) CometBroadcastExchange +(67) CometBroadcastExchange Input [1]: [i_item_sk#106] Arguments: [i_item_sk#106] -(64) CometBroadcastHashJoin +(68) 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 -(65) CometProject +(69) 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] -(66) ColumnarToRow [codegen id : 5] +(70) ColumnarToRow [codegen id : 5] Input [4]: [agg1#107, agg2#108, agg3#109, agg4#110] -(67) HashAggregate [codegen id : 5] +(71) HashAggregate [codegen id : 5] Input [4]: [agg1#107, agg2#108, agg3#109, agg4#110] Keys: [] 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] -(68) Exchange +(72) RowToColumnar +Input [8]: [sum#119, count#120, sum#121, count#122, sum#123, count#124, sum#125, count#126] + +(73) CometColumnarExchange +Input [8]: [sum#119, count#120, sum#121, count#122, sum#123, count#124, sum#125, count#126] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(74) ColumnarToRow [codegen id : 6] 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] -(69) HashAggregate [codegen id : 6] +(75) 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#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] -(70) Union +(76) Union -(71) TakeOrderedAndProject +(77) 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 (76) -+- * ColumnarToRow (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan parquet spark_catalog.default.date_dim (72) +BroadcastExchange (82) ++- * ColumnarToRow (81) + +- CometProject (80) + +- CometFilter (79) + +- CometScan parquet spark_catalog.default.date_dim (78) -(72) Scan parquet spark_catalog.default.date_dim +(78) 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 -(73) CometFilter +(79) 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)) -(74) CometProject +(80) CometProject Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(75) ColumnarToRow [codegen id : 1] +(81) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(76) BroadcastExchange +(82) BroadcastExchange Input [1]: [d_date_sk#14] 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#56 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#56 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#101 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 54 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 50e246e15..fd4d4f82e 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 @@ -2,92 +2,98 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] Union 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 (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] - 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,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] - CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_state] #5 - CometFilter [s_store_sk,s_state] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id,s_state] #1 + RowToColumnar + 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] + 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,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id] + CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [cd_demo_sk] #3 + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [s_store_sk,s_state] #5 + CometFilter [s_store_sk,s_state] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] + 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] #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] - 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,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] - CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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 - ReusedExchange [cd_demo_sk] #3 - ReusedExchange [d_date_sk] #4 - CometBroadcastExchange [s_store_sk] #8 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [i_item_sk,i_item_id] #6 + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #7 + RowToColumnar + 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] + 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,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] + CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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 + ReusedExchange [cd_demo_sk] #3 + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange [s_store_sk] #8 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + 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 #9 - WholeStageCodegen (5) - HashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - ColumnarToRow - InputAdapter - CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk] - CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] - CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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 - ReusedExchange [cd_demo_sk] #3 - ReusedExchange [d_date_sk] #4 - ReusedExchange [s_store_sk] #8 - CometBroadcastExchange [i_item_sk] #10 - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk] + ColumnarToRow + InputAdapter + CometColumnarExchange #9 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + ColumnarToRow + InputAdapter + CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [agg1,agg2,agg3,agg4] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk] + CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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 + ReusedExchange [cd_demo_sk] #3 + ReusedExchange [d_date_sk] #4 + ReusedExchange [s_store_sk] #8 + CometBroadcastExchange [i_item_sk] #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 92e11aabe..d978de885 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,37 +1,36 @@ == Physical Plan == -* 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) +* ColumnarToRow (32) ++- CometSort (31) + +- CometColumnarExchange (30) + +- CometProject (29) + +- CometBroadcastHashJoin (28) + :- CometFilter (24) + : +- CometHashAggregate (23) + : +- CometColumnarExchange (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) + +- CometBroadcastExchange (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.customer (25) (1) Scan parquet spark_catalog.default.store_sales @@ -135,90 +134,84 @@ 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)] -(22) ColumnarToRow [codegen id : 1] +(22) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(24) HashAggregate [codegen id : 3] +(23) CometHashAggregate 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)#17] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#17 AS cnt#18] -(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)) +(24) CometFilter +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] +Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) -(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] +(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] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(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) +(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) -(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] +(27) CometBroadcastExchange +Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Arguments: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -(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] +(28) CometBroadcastHashJoin +Left output [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] +Right output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight -(30) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#19] -Join type: Inner -Join condition: None +(29) CometProject +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] +Arguments: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17], [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -(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] +(30) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] -(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] +(31) CometSort +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, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17], [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] -(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 +(32) ColumnarToRow [codegen id : 1] +Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (38) -+- * ColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.date_dim (34) +BroadcastExchange (37) ++- * ColumnarToRow (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) -(34) Scan parquet spark_catalog.default.date_dim +(33) 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 -(35) CometFilter +(34) 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)) -(36) CometProject +(35) CometProject Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(37) ColumnarToRow [codegen id : 1] +(36) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(38) BroadcastExchange +(37) BroadcastExchange Input [1]: [d_date_sk#7] -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/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt index 814e543ed..fafc612d6 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,50 +1,42 @@ -WholeStageCodegen (4) - Sort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,count] - CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] - CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] - CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #5 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange [hd_demo_sk] #6 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 + CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometFilter [ss_ticket_number,ss_customer_sk,cnt] + CometHashAggregate [ss_ticket_number,ss_customer_sk,cnt,count,count(1)] + CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] + CometProject [ss_customer_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange [s_store_sk] #5 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_county] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county] + CometBroadcastExchange [hd_demo_sk] #6 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] #7 + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] 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 790000085..721036bfb 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,48 +1,50 @@ == Physical Plan == -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) +TakeOrderedAndProject (46) ++- * HashAggregate (45) + +- * ColumnarToRow (44) + +- CometColumnarExchange (43) + +- RowToColumnar (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 @@ -238,50 +240,56 @@ Functions [10]: [partial_count(1), partial_avg(cd_dep_count#25), partial_max(cd_ 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] -(42) Exchange +(42) RowToColumnar 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] -(43) HashAggregate [codegen id : 6] +(43) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=5] + +(44) ColumnarToRow [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] + +(45) 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] -(44) TakeOrderedAndProject +(46) 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 = 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) +BroadcastExchange (51) ++- * ColumnarToRow (50) + +- CometProject (49) + +- CometFilter (48) + +- CometScan parquet spark_catalog.default.date_dim (47) -(45) Scan parquet spark_catalog.default.date_dim +(47) 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 -(46) CometFilter +(48) 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)) -(47) CometProject +(49) CometProject Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(48) ColumnarToRow [codegen id : 1] +(50) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(49) BroadcastExchange +(51) BroadcastExchange Input [1]: [d_date_sk#9] 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/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/simplified.txt index 0b55e23ab..221648657 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,70 +1,72 @@ 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 (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 (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] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] - CometFilter [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] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,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 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + RowToColumnar + 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] + Project [c_current_cdemo_sk,ca_state] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] + Filter [exists,exists] + BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,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 + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [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] + CometBroadcastExchange [ss_customer_sk] #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,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 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [cs_ship_customer_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,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 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) + BroadcastExchange #8 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - 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] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + 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/q35a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/explain.txt index 8eafc64ff..675f2ac4b 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,42 +1,44 @@ == Physical Plan == -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) +TakeOrderedAndProject (40) ++- * HashAggregate (39) + +- * ColumnarToRow (38) + +- CometColumnarExchange (37) + +- RowToColumnar (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 @@ -207,50 +209,56 @@ Functions [10]: [partial_count(1), partial_avg(cd_dep_count#25), partial_max(cd_ 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 +(36) RowToColumnar 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] +(37) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(38) ColumnarToRow [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] + +(39) 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 +(40) 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 = 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) +BroadcastExchange (45) ++- * ColumnarToRow (44) + +- CometProject (43) + +- CometFilter (42) + +- CometScan parquet spark_catalog.default.date_dim (41) -(39) Scan parquet spark_catalog.default.date_dim +(41) 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 -(40) CometFilter +(42) 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)) -(41) CometProject +(43) CometProject Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(42) ColumnarToRow [codegen id : 1] +(44) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(43) BroadcastExchange +(45) BroadcastExchange Input [1]: [d_date_sk#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-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 79ead0464..c31cfd8da 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,52 +1,54 @@ 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 (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 (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] - 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,ca_state,cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometProject [c_current_cdemo_sk,ca_state] - CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk,ca_state] - CometProject [c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customsk] - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] - CometFilter [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] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [customsk] #5 - CometUnion [customsk] - CometProject [ws_bill_customer_sk] [customsk] - CometBroadcastHashJoin [ws_bill_customer_sk,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_ship_customer_sk,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 [ca_address_sk,ca_state] #6 - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + RowToColumnar + 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] + 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,ca_state,cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometProject [c_current_cdemo_sk,ca_state] + CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk,ca_state] + CometProject [c_current_cdemo_sk,c_current_addr_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customsk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [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] + CometBroadcastExchange [ss_customer_sk] #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [customsk] #5 + CometUnion [customsk] + CometProject [ws_bill_customer_sk] [customsk] + CometBroadcastHashJoin [ws_bill_customer_sk,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_ship_customer_sk,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 [ca_address_sk,ca_state] #6 + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + 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 7b3ae5c67..df24de6b8 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,46 +1,56 @@ == Physical Plan == -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) +TakeOrderedAndProject (52) ++- * Project (51) + +- Window (50) + +- * ColumnarToRow (49) + +- CometSort (48) + +- CometColumnarExchange (47) + +- CometHashAggregate (46) + +- CometColumnarExchange (45) + +- RowToColumnar (44) + +- * HashAggregate (43) + +- Union (42) + :- * HashAggregate (25) + : +- * ColumnarToRow (24) + : +- CometColumnarExchange (23) + : +- RowToColumnar (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 (33) + : +- * ColumnarToRow (32) + : +- CometColumnarExchange (31) + : +- RowToColumnar (30) + : +- * HashAggregate (29) + : +- * HashAggregate (28) + : +- * ColumnarToRow (27) + : +- ReusedExchange (26) + +- * HashAggregate (41) + +- * ColumnarToRow (40) + +- CometColumnarExchange (39) + +- RowToColumnar (38) + +- * HashAggregate (37) + +- * HashAggregate (36) + +- * ColumnarToRow (35) + +- ReusedExchange (34) (1) Scan parquet spark_catalog.default.store_sales @@ -145,142 +155,170 @@ Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(Unscale Aggregate Attributes [2]: [sum#14, sum#15] Results [4]: [i_category#11, i_class#10, sum#16, sum#17] -(22) Exchange +(22) RowToColumnar 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] -(23) HashAggregate [codegen id : 2] +(23) CometColumnarExchange +Input [4]: [i_category#11, i_class#10, sum#16, sum#17] +Arguments: hashpartitioning(i_category#11, i_class#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(24) ColumnarToRow [codegen id : 2] +Input [4]: [i_category#11, i_class#10, sum#16, sum#17] + +(25) 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))#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] +(26) ReusedExchange [Reuses operator id: 23] Output [4]: [i_category#24, i_class#25, sum#26, sum#27] -(25) HashAggregate [codegen id : 4] +(27) ColumnarToRow [codegen id : 4] +Input [4]: [i_category#24, i_class#25, sum#26, sum#27] + +(28) 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] +(29) 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 +(30) RowToColumnar 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] +(31) CometColumnarExchange +Input [5]: [i_category#24, sum#38, isEmpty#39, sum#40, isEmpty#41] +Arguments: hashpartitioning(i_category#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(32) ColumnarToRow [codegen id : 5] +Input [5]: [i_category#24, sum#38, isEmpty#39, sum#40, isEmpty#41] + +(33) 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] +(34) ReusedExchange [Reuses operator id: 23] Output [4]: [i_category#49, i_class#50, sum#51, sum#52] -(30) HashAggregate [codegen id : 7] +(35) ColumnarToRow [codegen id : 7] +Input [4]: [i_category#49, i_class#50, sum#51, sum#52] + +(36) 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] +(37) HashAggregate [codegen id : 7] Input [2]: [ss_net_profit#55, ss_ext_sales_price#56] Keys: [] 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] -(32) Exchange +(38) RowToColumnar +Input [4]: [sum#61, isEmpty#62, sum#63, isEmpty#64] + +(39) CometColumnarExchange +Input [4]: [sum#61, isEmpty#62, sum#63, isEmpty#64] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(40) ColumnarToRow [codegen id : 8] Input [4]: [sum#61, isEmpty#62, sum#63, isEmpty#64] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] -(33) HashAggregate [codegen id : 8] +(41) HashAggregate [codegen id : 8] Input [4]: [sum#61, isEmpty#62, sum#63, isEmpty#64] Keys: [] 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] -(34) Union +(42) Union -(35) HashAggregate [codegen id : 9] +(43) 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#20, i_category#11, i_class#10, t_category#21, t_class#22, lochierarchy#23] -(36) Exchange +(44) RowToColumnar 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] -(37) HashAggregate [codegen id : 10] +(45) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=4] + +(46) CometHashAggregate 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#20, i_category#11, i_class#10, lochierarchy#23, CASE WHEN (t_class#22 = 0) THEN i_category#11 END AS _w0#73] -(38) Exchange +(47) CometColumnarExchange +Input [5]: [gross_margin#20, i_category#11, i_class#10, lochierarchy#23, _w0#73] +Arguments: hashpartitioning(lochierarchy#23, _w0#73, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(48) CometSort 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] +Arguments: [gross_margin#20, i_category#11, i_class#10, lochierarchy#23, _w0#73], [lochierarchy#23 ASC NULLS FIRST, _w0#73 ASC NULLS FIRST, gross_margin#20 ASC NULLS FIRST] -(39) Sort [codegen id : 11] +(49) ColumnarToRow [codegen id : 10] 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 -(40) Window +(50) 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] -(41) Project [codegen id : 12] +(51) Project [codegen id : 11] 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] -(42) TakeOrderedAndProject +(52) 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 (47) -+- * ColumnarToRow (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan parquet spark_catalog.default.date_dim (43) +BroadcastExchange (57) ++- * ColumnarToRow (56) + +- CometProject (55) + +- CometFilter (54) + +- CometScan parquet spark_catalog.default.date_dim (53) -(43) Scan parquet spark_catalog.default.date_dim +(53) 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 -(44) CometFilter +(54) 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)) -(45) CometProject +(55) CometProject Input [2]: [d_date_sk#7, d_year#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(46) ColumnarToRow [codegen id : 1] +(56) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(47) BroadcastExchange +(57) BroadcastExchange Input [1]: [d_date_sk#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/q36a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt index 59c22c335..d2c385a14 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,70 +1,78 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (12) + WholeStageCodegen (11) Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [gross_margin,lochierarchy,_w0] - WholeStageCodegen (11) - Sort [lochierarchy,_w0,gross_margin] + WholeStageCodegen (10) + ColumnarToRow InputAdapter - Exchange [lochierarchy,_w0] #1 - 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 + CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] + CometColumnarExchange [lochierarchy,_w0] #1 + CometHashAggregate [gross_margin,i_category,i_class,lochierarchy,_w0,t_category,t_class] + CometColumnarExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 + RowToColumnar WholeStageCodegen (9) HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] InputAdapter Union 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 (1) - HashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] - ColumnarToRow - InputAdapter - CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] - CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] - CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_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 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometFilter [i_item_sk,i_class,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #7 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class] #3 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] + ColumnarToRow + InputAdapter + CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] + CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_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 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometFilter [i_item_sk,i_class,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometBroadcastExchange [s_store_sk] #7 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + 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] #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 + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category] #8 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,sum,sum] #3 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 #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 - ReusedExchange [i_category,i_class,sum,sum] #3 + ColumnarToRow + InputAdapter + CometColumnarExchange #9 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,sum,sum] #3 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 81588ca21..bd9529104 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 @@ -1,49 +1,57 @@ == Physical Plan == -TakeOrderedAndProject (45) -+- * Project (44) - +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (28) - : : +- * Filter (27) - : : +- Window (26) - : : +- * Filter (25) - : : +- Window (24) - : : +- * Sort (23) - : : +- Exchange (22) - : : +- * HashAggregate (21) - : : +- Exchange (20) - : : +- * HashAggregate (19) - : : +- * 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) - : +- * Project (34) - : +- Window (33) - : +- * Sort (32) - : +- Exchange (31) - : +- * HashAggregate (30) - : +- ReusedExchange (29) - +- BroadcastExchange (42) - +- * Project (41) - +- Window (40) - +- * Sort (39) - +- ReusedExchange (38) +TakeOrderedAndProject (53) ++- * Project (52) + +- * BroadcastHashJoin Inner BuildRight (51) + :- * Project (44) + : +- * BroadcastHashJoin Inner BuildRight (43) + : :- * Project (32) + : : +- * Filter (31) + : : +- Window (30) + : : +- * Filter (29) + : : +- Window (28) + : : +- * ColumnarToRow (27) + : : +- CometSort (26) + : : +- CometColumnarExchange (25) + : : +- RowToColumnar (24) + : : +- * HashAggregate (23) + : : +- * ColumnarToRow (22) + : : +- CometColumnarExchange (21) + : : +- RowToColumnar (20) + : : +- * HashAggregate (19) + : : +- * 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 (42) + : +- * Project (41) + : +- Window (40) + : +- * ColumnarToRow (39) + : +- CometSort (38) + : +- CometColumnarExchange (37) + : +- RowToColumnar (36) + : +- * HashAggregate (35) + : +- * ColumnarToRow (34) + : +- ReusedExchange (33) + +- BroadcastExchange (50) + +- * Project (49) + +- Window (48) + +- * ColumnarToRow (47) + +- CometSort (46) + +- ReusedExchange (45) (1) Scan parquet spark_catalog.default.item @@ -140,142 +148,166 @@ 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] -(20) Exchange +(20) RowToColumnar 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=1] -(21) HashAggregate [codegen id : 2] +(21) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(22) ColumnarToRow [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] + +(23) 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))] 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] -(22) Exchange +(24) RowToColumnar +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] + +(25) CometColumnarExchange 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=2] +Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(23) Sort [codegen id : 3] +(26) CometSort 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 +Arguments: [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], [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] -(24) Window +(27) ColumnarToRow [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] + +(28) 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] -(25) Filter [codegen id : 4] +(29) 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)) -(26) Window +(30) 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] -(27) Filter [codegen id : 13] +(31) 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 : 13] +(32) 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] +(33) ReusedExchange [Reuses operator id: 21] 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 : 6] +(34) ColumnarToRow [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] + +(35) 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))] 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#30] -(31) Exchange +(36) RowToColumnar +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] + +(37) CometColumnarExchange 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=3] +Arguments: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(32) Sort [codegen id : 7] +(38) CometSort 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 +Arguments: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30], [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] -(33) Window +(39) ColumnarToRow [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] + +(40) 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#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 : 8] +(41) 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 +(42) 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=4] -(36) BroadcastHashJoin [codegen id : 13] +(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#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 : 13] +(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, 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] +(45) ReusedExchange [Reuses operator id: 37] 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 : 11] +(46) CometSort 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 +Arguments: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38], [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] -(40) Window +(47) ColumnarToRow [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] + +(48) 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#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 : 12] +(49) 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 +(50) 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=5] -(43) BroadcastHashJoin [codegen id : 13] +(51) 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 : 13] +(52) 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] -(45) TakeOrderedAndProject +(53) 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] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (49) -+- * ColumnarToRow (48) - +- CometFilter (47) - +- CometScan parquet spark_catalog.default.date_dim (46) +BroadcastExchange (57) ++- * ColumnarToRow (56) + +- CometFilter (55) + +- CometScan parquet spark_catalog.default.date_dim (54) -(46) Scan parquet spark_catalog.default.date_dim +(54) 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 -(47) CometFilter +(55) 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)) -(48) ColumnarToRow [codegen id : 1] +(56) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -(49) BroadcastExchange +(57) 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=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 61f4976b4..163948e42 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 @@ -13,41 +13,45 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (3) - Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + ColumnarToRow InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name] #1 - 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 (1) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometFilter [i_item_sk,i_brand,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #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] + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #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 #7 WholeStageCodegen (8) @@ -55,13 +59,16 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (7) - Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + ColumnarToRow InputAdapter - 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 + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #8 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 InputAdapter BroadcastExchange #9 WholeStageCodegen (12) @@ -69,6 +76,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (11) - Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + ColumnarToRow InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + 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 01c4c5583..895ed6510 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,81 +1,95 @@ == Physical Plan == -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) +* ColumnarToRow (91) ++- CometTakeOrderedAndProject (90) + +- CometHashAggregate (89) + +- CometColumnarExchange (88) + +- RowToColumnar (87) + +- * HashAggregate (86) + +- Union (85) + :- * Project (30) + : +- * Filter (29) + : +- Window (28) + : +- * Sort (27) + : +- Window (26) + : +- * ColumnarToRow (25) + : +- CometSort (24) + : +- CometColumnarExchange (23) + : +- RowToColumnar (22) + : +- * HashAggregate (21) + : +- * ColumnarToRow (20) + : +- CometColumnarExchange (19) + : +- RowToColumnar (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 (57) + : +- * Filter (56) + : +- Window (55) + : +- * Sort (54) + : +- Window (53) + : +- * ColumnarToRow (52) + : +- CometSort (51) + : +- CometColumnarExchange (50) + : +- RowToColumnar (49) + : +- * HashAggregate (48) + : +- * ColumnarToRow (47) + : +- CometColumnarExchange (46) + : +- RowToColumnar (45) + : +- * HashAggregate (44) + : +- * ColumnarToRow (43) + : +- CometProject (42) + : +- CometBroadcastHashJoin (41) + : :- CometProject (39) + : : +- CometBroadcastHashJoin (38) + : : :- CometBroadcastExchange (34) + : : : +- CometProject (33) + : : : +- CometFilter (32) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (31) + : : +- CometProject (37) + : : +- CometFilter (36) + : : +- CometScan parquet spark_catalog.default.catalog_returns (35) + : +- ReusedExchange (40) + +- * Project (84) + +- * Filter (83) + +- Window (82) + +- * Sort (81) + +- Window (80) + +- * ColumnarToRow (79) + +- CometSort (78) + +- CometColumnarExchange (77) + +- RowToColumnar (76) + +- * HashAggregate (75) + +- * ColumnarToRow (74) + +- CometColumnarExchange (73) + +- RowToColumnar (72) + +- * HashAggregate (71) + +- * ColumnarToRow (70) + +- CometProject (69) + +- CometBroadcastHashJoin (68) + :- CometProject (66) + : +- CometBroadcastHashJoin (65) + : :- CometBroadcastExchange (61) + : : +- CometProject (60) + : : +- CometFilter (59) + : : +- CometScan parquet spark_catalog.default.store_sales (58) + : +- CometProject (64) + : +- CometFilter (63) + : +- CometScan parquet spark_catalog.default.store_returns (62) + +- ReusedExchange (67) (1) Scan parquet spark_catalog.default.web_sales @@ -160,46 +174,58 @@ Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coa 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] -(18) Exchange +(18) RowToColumnar 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] -(19) HashAggregate [codegen id : 2] +(19) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(20) ColumnarToRow [codegen id : 2] +Input [7]: [ws_item_sk#1, sum#22, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] + +(21) 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))#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] -(20) Exchange +(22) RowToColumnar Input [3]: [item#32, return_ratio#33, currency_ratio#34] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] -(21) Sort [codegen id : 3] +(23) CometColumnarExchange Input [3]: [item#32, return_ratio#33, currency_ratio#34] -Arguments: [return_ratio#33 ASC NULLS FIRST], false, 0 +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(22) Window +(24) CometSort +Input [3]: [item#32, return_ratio#33, currency_ratio#34] +Arguments: [item#32, return_ratio#33, currency_ratio#34], [return_ratio#33 ASC NULLS FIRST] + +(25) ColumnarToRow [codegen id : 3] +Input [3]: [item#32, return_ratio#33, currency_ratio#34] + +(26) 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] -(23) Sort [codegen id : 4] +(27) 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 -(24) Window +(28) 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] -(25) Filter [codegen id : 5] +(29) 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)) -(26) Project [codegen id : 5] +(30) 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] -(27) Scan parquet spark_catalog.default.catalog_sales +(31) 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 [] @@ -207,104 +233,116 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_so 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 -(28) CometFilter +(32) 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)) -(29) CometProject +(33) 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] -(30) CometBroadcastExchange +(34) 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] -(31) Scan parquet spark_catalog.default.catalog_returns +(35) 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 -(32) CometFilter +(36) 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 +(37) 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 +(38) 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 +(39) 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] +(40) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#50] -(37) CometBroadcastHashJoin +(41) 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 +(42) 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] +(43) 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] +(44) 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 +(45) RowToColumnar +Input [7]: [cs_item_sk#38, sum#57, sum#58, sum#59, isEmpty#60, sum#61, isEmpty#62] + +(46) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=3] + +(47) ColumnarToRow [codegen id : 7] 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] +(48) 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 +(49) RowToColumnar +Input [3]: [item#67, return_ratio#68, currency_ratio#69] + +(50) CometColumnarExchange +Input [3]: [item#67, return_ratio#68, currency_ratio#69] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(51) CometSort Input [3]: [item#67, return_ratio#68, currency_ratio#69] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: [item#67, return_ratio#68, currency_ratio#69], [return_ratio#68 ASC NULLS FIRST] -(44) Sort [codegen id : 8] +(52) ColumnarToRow [codegen id : 8] Input [3]: [item#67, return_ratio#68, currency_ratio#69] -Arguments: [return_ratio#68 ASC NULLS FIRST], false, 0 -(45) Window +(53) 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] -(46) Sort [codegen id : 9] +(54) 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 -(47) Window +(55) 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] -(48) Filter [codegen id : 10] +(56) 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)) -(49) Project [codegen id : 10] +(57) 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] -(50) Scan parquet spark_catalog.default.store_sales +(58) 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 [] @@ -312,161 +350,177 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#78), dynamicpruningexpression(ss_so 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 -(51) CometFilter +(59) 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)) -(52) CometProject +(60) 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] -(53) CometBroadcastExchange +(61) 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] -(54) Scan parquet spark_catalog.default.store_returns +(62) 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 -(55) CometFilter +(63) 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 +(64) 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 +(65) 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 +(66) 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] +(67) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#85] -(60) CometBroadcastHashJoin +(68) 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 +(69) 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] +(70) 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] +(71) 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 +(72) RowToColumnar +Input [7]: [ss_item_sk#73, sum#92, sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] + +(73) CometColumnarExchange 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] +Arguments: hashpartitioning(ss_item_sk#73, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(65) HashAggregate [codegen id : 12] +(74) ColumnarToRow [codegen id : 12] +Input [7]: [ss_item_sk#73, sum#92, sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] + +(75) 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 +(76) RowToColumnar +Input [3]: [item#102, return_ratio#103, currency_ratio#104] + +(77) CometColumnarExchange Input [3]: [item#102, return_ratio#103, currency_ratio#104] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(67) Sort [codegen id : 13] +(78) CometSort Input [3]: [item#102, return_ratio#103, currency_ratio#104] -Arguments: [return_ratio#103 ASC NULLS FIRST], false, 0 +Arguments: [item#102, return_ratio#103, currency_ratio#104], [return_ratio#103 ASC NULLS FIRST] -(68) Window +(79) ColumnarToRow [codegen id : 13] +Input [3]: [item#102, return_ratio#103, currency_ratio#104] + +(80) 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] -(69) Sort [codegen id : 14] +(81) 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 -(70) Window +(82) 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] -(71) Filter [codegen id : 15] +(83) 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)) -(72) Project [codegen id : 15] +(84) 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] -(73) Union +(85) Union -(74) HashAggregate [codegen id : 16] +(86) 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#37, item#32, return_ratio#33, return_rank#35, currency_rank#36] -(75) Exchange +(87) RowToColumnar +Input [5]: [channel#37, item#32, return_ratio#33, return_rank#35, currency_rank#36] + +(88) CometColumnarExchange 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] +Arguments: hashpartitioning(channel#37, item#32, return_ratio#33, return_rank#35, currency_rank#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(76) HashAggregate [codegen id : 17] +(89) CometHashAggregate 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#37, item#32, return_ratio#33, return_rank#35, currency_rank#36] -(77) TakeOrderedAndProject +(90) CometTakeOrderedAndProject +Input [5]: [channel#37, item#32, return_ratio#33, return_rank#35, currency_rank#36] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#37 ASC NULLS FIRST,return_rank#35 ASC NULLS FIRST,currency_rank#36 ASC NULLS FIRST,item#32 ASC NULLS FIRST], output=[channel#37,item#32,return_ratio#33,return_rank#35,currency_rank#36]), [channel#37, item#32, return_ratio#33, return_rank#35, currency_rank#36], 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] + +(91) ColumnarToRow [codegen id : 17] 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 (82) -+- * ColumnarToRow (81) - +- CometProject (80) - +- CometFilter (79) - +- CometScan parquet spark_catalog.default.date_dim (78) +BroadcastExchange (96) ++- * ColumnarToRow (95) + +- CometProject (94) + +- CometFilter (93) + +- CometScan parquet spark_catalog.default.date_dim (92) -(78) Scan parquet spark_catalog.default.date_dim +(92) 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 -(79) CometFilter +(93) 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)) -(80) CometProject +(94) CometProject Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(81) ColumnarToRow [codegen id : 1] +(95) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(82) BroadcastExchange +(96) 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 = 27 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 31 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#78 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 58 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 bd3b21cdb..4d4e531a0 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,121 +1,135 @@ -TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] - WholeStageCodegen (17) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Exchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (16) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (5) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (4) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (3) - Sort [return_ratio] - InputAdapter - Exchange #2 - 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 (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] +WholeStageCodegen (17) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] + CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] + CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 + RowToColumnar + WholeStageCodegen (16) + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] + InputAdapter + Union + WholeStageCodegen (5) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (4) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #2 + RowToColumnar + 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] ColumnarToRow InputAdapter - CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - 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 (9) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (8) - Sort [return_ratio] - InputAdapter - 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] #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] + CometColumnarExchange [ws_item_sk] #3 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + 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 (9) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #7 + RowToColumnar + 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] ColumnarToRow InputAdapter - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_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] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - 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 (14) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (13) - Sort [return_ratio] - InputAdapter - 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] #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] + CometColumnarExchange [cs_item_sk] #8 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_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] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + 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 (14) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (13) + ColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #10 + RowToColumnar + 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] ColumnarToRow InputAdapter - CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_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] - ReusedSubquery [d_date_sk] #1 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - ReusedExchange [d_date_sk] #6 + CometColumnarExchange [ss_item_sk] #11 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_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] + ReusedSubquery [d_date_sk] #1 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + 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 a5ad8c293..b92d21780 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,74 +1,92 @@ == Physical Plan == -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) +TakeOrderedAndProject (88) ++- * Filter (87) + +- * HashAggregate (86) + +- * HashAggregate (85) + +- * Project (84) + +- * BroadcastHashJoin Inner BuildRight (83) + :- Window (76) + : +- * ColumnarToRow (75) + : +- CometSort (74) + : +- CometColumnarExchange (73) + : +- CometProject (72) + : +- CometFilter (71) + : +- CometSortMergeJoin (70) + : :- CometSort (36) + : : +- CometColumnarExchange (35) + : : +- RowToColumnar (34) + : : +- * HashAggregate (33) + : : +- * ColumnarToRow (32) + : : +- CometColumnarExchange (31) + : : +- RowToColumnar (30) + : : +- * HashAggregate (29) + : : +- * Project (28) + : : +- * BroadcastHashJoin Inner BuildRight (27) + : : :- * Project (20) + : : : +- Window (19) + : : : +- * ColumnarToRow (18) + : : : +- CometSort (17) + : : : +- CometColumnarExchange (16) + : : : +- RowToColumnar (15) + : : : +- * HashAggregate (14) + : : : +- * ColumnarToRow (13) + : : : +- CometColumnarExchange (12) + : : : +- RowToColumnar (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 (26) + : : +- * Project (25) + : : +- Window (24) + : : +- * ColumnarToRow (23) + : : +- CometSort (22) + : : +- ReusedExchange (21) + : +- CometSort (69) + : +- CometColumnarExchange (68) + : +- RowToColumnar (67) + : +- * HashAggregate (66) + : +- * ColumnarToRow (65) + : +- CometColumnarExchange (64) + : +- RowToColumnar (63) + : +- * HashAggregate (62) + : +- * Project (61) + : +- * BroadcastHashJoin Inner BuildRight (60) + : :- * Project (53) + : : +- Window (52) + : : +- * ColumnarToRow (51) + : : +- CometSort (50) + : : +- CometColumnarExchange (49) + : : +- RowToColumnar (48) + : : +- * HashAggregate (47) + : : +- * ColumnarToRow (46) + : : +- CometColumnarExchange (45) + : : +- RowToColumnar (44) + : : +- * HashAggregate (43) + : : +- * ColumnarToRow (42) + : : +- CometProject (41) + : : +- CometBroadcastHashJoin (40) + : : :- CometFilter (38) + : : : +- CometScan parquet spark_catalog.default.store_sales (37) + : : +- ReusedExchange (39) + : +- BroadcastExchange (59) + : +- * Project (58) + : +- Window (57) + : +- * ColumnarToRow (56) + : +- CometSort (55) + : +- ReusedExchange (54) + +- BroadcastExchange (82) + +- * Project (81) + +- Window (80) + +- * ColumnarToRow (79) + +- CometSort (78) + +- ReusedExchange (77) (1) Scan parquet spark_catalog.default.web_sales @@ -121,89 +139,113 @@ Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] Aggregate Attributes [1]: [sum#8] Results [3]: [ws_item_sk#1, d_date#6, sum#9] -(11) Exchange +(11) RowToColumnar 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] -(12) HashAggregate [codegen id : 2] +(12) CometColumnarExchange +Input [3]: [ws_item_sk#1, d_date#6, sum#9] +Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(13) ColumnarToRow [codegen id : 2] +Input [3]: [ws_item_sk#1, d_date#6, sum#9] + +(14) 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))#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] -(13) Exchange +(15) RowToColumnar +Input [4]: [item_sk#11, d_date#6, sumws#12, ws_item_sk#1] + +(16) CometColumnarExchange 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] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(14) Sort [codegen id : 3] +(17) CometSort 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 +Arguments: [item_sk#11, d_date#6, sumws#12, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(15) Window +(18) ColumnarToRow [codegen id : 3] +Input [4]: [item_sk#11, d_date#6, sumws#12, ws_item_sk#1] + +(19) 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] -(16) Project [codegen id : 8] +(20) 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] -(17) ReusedExchange [Reuses operator id: 13] +(21) ReusedExchange [Reuses operator id: 16] Output [4]: [item_sk#14, d_date#15, sumws#16, ws_item_sk#17] -(18) Sort [codegen id : 6] +(22) CometSort 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 +Arguments: [item_sk#14, d_date#15, sumws#16, ws_item_sk#17], [ws_item_sk#17 ASC NULLS FIRST, d_date#15 ASC NULLS FIRST] -(19) Window +(23) ColumnarToRow [codegen id : 6] +Input [4]: [item_sk#14, d_date#15, sumws#16, ws_item_sk#17] + +(24) 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] -(20) Project [codegen id : 7] +(25) 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] -(21) BroadcastExchange +(26) BroadcastExchange Input [3]: [item_sk#14, sumws#16, rk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(22) BroadcastHashJoin [codegen id : 8] +(27) BroadcastHashJoin [codegen id : 8] Left keys [1]: [item_sk#11] Right keys [1]: [item_sk#14] Join type: Inner Join condition: (rk#13 >= rk#18) -(23) Project [codegen id : 8] +(28) 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] +(29) 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 +(30) RowToColumnar 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] +(31) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=4] + +(32) ColumnarToRow [codegen id : 9] +Input [5]: [item_sk#11, d_date#6, sumws#12, sum#21, isEmpty#22] + +(33) 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 +(34) RowToColumnar +Input [3]: [item_sk#11, d_date#6, cume_sales#24] + +(35) CometColumnarExchange 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] +Arguments: hashpartitioning(item_sk#11, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(28) Sort [codegen id : 10] +(36) CometSort 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 +Arguments: [item_sk#11, d_date#6, cume_sales#24], [item_sk#11 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(29) Scan parquet spark_catalog.default.store_sales +(37) 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 [] @@ -211,223 +253,252 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#27), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(30) CometFilter +(38) CometFilter Input [3]: [ss_item_sk#25, ss_sales_price#26, ss_sold_date_sk#27] Condition : isnotnull(ss_item_sk#25) -(31) ReusedExchange [Reuses operator id: 6] +(39) ReusedExchange [Reuses operator id: 6] Output [2]: [d_date_sk#29, d_date#30] -(32) CometBroadcastHashJoin +(40) 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 -(33) CometProject +(41) 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] -(34) ColumnarToRow [codegen id : 11] +(42) ColumnarToRow [codegen id : 10] Input [3]: [ss_item_sk#25, ss_sales_price#26, d_date#30] -(35) HashAggregate [codegen id : 11] +(43) HashAggregate [codegen id : 10] 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 +(44) RowToColumnar +Input [3]: [ss_item_sk#25, d_date#30, sum#32] + +(45) CometColumnarExchange +Input [3]: [ss_item_sk#25, d_date#30, sum#32] +Arguments: hashpartitioning(ss_item_sk#25, d_date#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(46) ColumnarToRow [codegen id : 11] 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] +(47) HashAggregate [codegen id : 11] 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 +(48) RowToColumnar +Input [4]: [item_sk#34, d_date#30, sumss#35, ss_item_sk#25] + +(49) CometColumnarExchange 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] +Arguments: hashpartitioning(ss_item_sk#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(39) Sort [codegen id : 13] +(50) CometSort 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 +Arguments: [item_sk#34, d_date#30, sumss#35, ss_item_sk#25], [ss_item_sk#25 ASC NULLS FIRST, d_date#30 ASC NULLS FIRST] -(40) Window +(51) ColumnarToRow [codegen id : 12] +Input [4]: [item_sk#34, d_date#30, sumss#35, ss_item_sk#25] + +(52) 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] +(53) Project [codegen id : 17] 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] +(54) ReusedExchange [Reuses operator id: 49] Output [4]: [item_sk#37, d_date#38, sumss#39, ss_item_sk#40] -(43) Sort [codegen id : 16] +(55) CometSort 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 +Arguments: [item_sk#37, d_date#38, sumss#39, ss_item_sk#40], [ss_item_sk#40 ASC NULLS FIRST, d_date#38 ASC NULLS FIRST] -(44) Window +(56) ColumnarToRow [codegen id : 15] +Input [4]: [item_sk#37, d_date#38, sumss#39, ss_item_sk#40] + +(57) 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] +(58) Project [codegen id : 16] 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 +(59) BroadcastExchange Input [3]: [item_sk#37, sumss#39, rk#41] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -(47) BroadcastHashJoin [codegen id : 18] +(60) BroadcastHashJoin [codegen id : 17] Left keys [1]: [item_sk#34] Right keys [1]: [item_sk#37] Join type: Inner Join condition: (rk#36 >= rk#41) -(48) Project [codegen id : 18] +(61) Project [codegen id : 17] 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] +(62) HashAggregate [codegen id : 17] 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 +(63) RowToColumnar 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] +(64) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=9] + +(65) ColumnarToRow [codegen id : 18] +Input [5]: [item_sk#34, d_date#30, sumss#35, sum#44, isEmpty#45] + +(66) HashAggregate [codegen id : 18] 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 +(67) RowToColumnar +Input [3]: [item_sk#34, d_date#30, cume_sales#47] + +(68) CometColumnarExchange 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] +Arguments: hashpartitioning(item_sk#34, d_date#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(53) Sort [codegen id : 20] +(69) CometSort 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 +Arguments: [item_sk#34, d_date#30, cume_sales#47], [item_sk#34 ASC NULLS FIRST, d_date#30 ASC NULLS FIRST] -(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 +(70) CometSortMergeJoin +Left output [3]: [item_sk#11, d_date#6, cume_sales#24] +Right output [3]: [item_sk#34, d_date#30, cume_sales#47] +Arguments: [item_sk#11, d_date#6], [item_sk#34, d_date#30], FullOuter -(55) Filter [codegen id : 21] +(71) CometFilter 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) -(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] +(72) CometProject Input [6]: [item_sk#11, d_date#6, cume_sales#24, item_sk#34, d_date#30, cume_sales#47] +Arguments: [item_sk#48, d_date#49, web_sales#50, store_sales#51], [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] -(57) Exchange +(73) CometColumnarExchange 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] +Arguments: hashpartitioning(item_sk#48, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(58) Sort [codegen id : 22] +(74) CometSort 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 +Arguments: [item_sk#48, d_date#49, web_sales#50, store_sales#51], [item_sk#48 ASC NULLS FIRST, d_date#49 ASC NULLS FIRST] -(59) Window +(75) ColumnarToRow [codegen id : 19] +Input [4]: [item_sk#48, d_date#49, web_sales#50, store_sales#51] + +(76) 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] -(60) ReusedExchange [Reuses operator id: 57] +(77) ReusedExchange [Reuses operator id: 73] Output [4]: [item_sk#53, d_date#54, web_sales#55, store_sales#56] -(61) Sort [codegen id : 44] +(78) CometSort +Input [4]: [item_sk#53, d_date#54, web_sales#55, store_sales#56] +Arguments: [item_sk#53, d_date#54, web_sales#55, store_sales#56], [item_sk#53 ASC NULLS FIRST, d_date#54 ASC NULLS FIRST] + +(79) ColumnarToRow [codegen id : 38] 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 -(62) Window +(80) 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] -(63) Project [codegen id : 45] +(81) Project [codegen id : 39] 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] -(64) BroadcastExchange +(82) 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] -(65) BroadcastHashJoin [codegen id : 46] +(83) BroadcastHashJoin [codegen id : 40] Left keys [1]: [item_sk#48] Right keys [1]: [item_sk#53] Join type: Inner Join condition: (rk#52 >= rk#57) -(66) Project [codegen id : 46] +(84) Project [codegen id : 40] 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] +(85) HashAggregate [codegen id : 40] 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] +(86) HashAggregate [codegen id : 40] 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] +(87) Filter [codegen id : 40] 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 +(88) 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 (75) -+- * ColumnarToRow (74) - +- CometProject (73) - +- CometFilter (72) - +- CometScan parquet spark_catalog.default.date_dim (71) +BroadcastExchange (93) ++- * ColumnarToRow (92) + +- CometProject (91) + +- CometFilter (90) + +- CometScan parquet spark_catalog.default.date_dim (89) -(71) Scan parquet spark_catalog.default.date_dim +(89) 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 -(72) CometFilter +(90) 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)) -(73) CometProject +(91) 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] -(74) ColumnarToRow [codegen id : 1] +(92) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(75) BroadcastExchange +(93) 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 = 29 Hosting Expression = ss_sold_date_sk#27 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 37 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 47ad5e34f..f7edf9b01 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 (46) + WholeStageCodegen (40) 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,23 +7,23 @@ 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 (22) - Sort [item_sk,d_date] + WholeStageCodegen (19) + ColumnarToRow InputAdapter - Exchange [item_sk] #1 - 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 (10) - Sort [item_sk,d_date] - InputAdapter - Exchange [item_sk,d_date] #2 - WholeStageCodegen (9) - HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] - InputAdapter - Exchange [item_sk,d_date,sumws] #3 + CometSort [item_sk,d_date,web_sales,store_sales] + CometColumnarExchange [item_sk] #1 + CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] + CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #2 + RowToColumnar + WholeStageCodegen (9) + HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [item_sk,d_date,sumws] #3 + RowToColumnar WholeStageCodegen (8) HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] Project [item_sk,d_date,sumws,sumws] @@ -32,33 +32,37 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store InputAdapter Window [ws_item_sk,d_date] WholeStageCodegen (3) - Sort [ws_item_sk,d_date] + ColumnarToRow InputAdapter - Exchange [ws_item_sk] #4 - 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 (1) - HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometSort [item_sk,d_date,sumws,ws_item_sk] + CometColumnarExchange [ws_item_sk] #4 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [ws_item_sk,d_date] #5 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [ws_item_sk,ws_sales_price,d_date] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange [d_date_sk,d_date] #7 + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter BroadcastExchange #8 WholeStageCodegen (7) @@ -66,60 +70,67 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store InputAdapter Window [ws_item_sk,d_date] WholeStageCodegen (6) - Sort [ws_item_sk,d_date] + ColumnarToRow InputAdapter - ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 - InputAdapter - WholeStageCodegen (20) - Sort [item_sk,d_date] - InputAdapter - 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] #10 - WholeStageCodegen (18) + CometSort [item_sk,d_date,sumws,ws_item_sk] + ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #9 + RowToColumnar + WholeStageCodegen (18) + HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [item_sk,d_date,sumss] #10 + RowToColumnar + WholeStageCodegen (17) 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 (13) - Sort [ss_item_sk,d_date] + WholeStageCodegen (12) + ColumnarToRow InputAdapter - 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] #12 - WholeStageCodegen (11) - HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_date] #7 + CometSort [item_sk,d_date,sumss,ss_item_sk] + CometColumnarExchange [ss_item_sk] #11 + RowToColumnar + WholeStageCodegen (11) + HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [ss_item_sk,d_date] #12 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_sales_price,d_date] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_date] #7 InputAdapter BroadcastExchange #13 - WholeStageCodegen (17) + WholeStageCodegen (16) Project [item_sk,sumss,rk] InputAdapter Window [ss_item_sk,d_date] - WholeStageCodegen (16) - Sort [ss_item_sk,d_date] + WholeStageCodegen (15) + ColumnarToRow InputAdapter - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 + CometSort [item_sk,d_date,sumss,ss_item_sk] + ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 InputAdapter BroadcastExchange #14 - WholeStageCodegen (45) + WholeStageCodegen (39) Project [item_sk,web_sales,store_sales,rk] InputAdapter Window [item_sk,d_date] - WholeStageCodegen (44) - Sort [item_sk,d_date] + WholeStageCodegen (38) + ColumnarToRow InputAdapter - ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 + CometSort [item_sk,d_date,web_sales,store_sales] + 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 e361df2d1..56702651b 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 @@ -1,49 +1,57 @@ == Physical Plan == -TakeOrderedAndProject (45) -+- * Project (44) - +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (28) - : : +- * Filter (27) - : : +- Window (26) - : : +- * Filter (25) - : : +- Window (24) - : : +- * Sort (23) - : : +- Exchange (22) - : : +- * HashAggregate (21) - : : +- Exchange (20) - : : +- * HashAggregate (19) - : : +- * 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) - : +- * Project (34) - : +- Window (33) - : +- * Sort (32) - : +- Exchange (31) - : +- * HashAggregate (30) - : +- ReusedExchange (29) - +- BroadcastExchange (42) - +- * Project (41) - +- Window (40) - +- * Sort (39) - +- ReusedExchange (38) +TakeOrderedAndProject (53) ++- * Project (52) + +- * BroadcastHashJoin Inner BuildRight (51) + :- * Project (44) + : +- * BroadcastHashJoin Inner BuildRight (43) + : :- * Project (32) + : : +- * Filter (31) + : : +- Window (30) + : : +- * Filter (29) + : : +- Window (28) + : : +- * ColumnarToRow (27) + : : +- CometSort (26) + : : +- CometColumnarExchange (25) + : : +- RowToColumnar (24) + : : +- * HashAggregate (23) + : : +- * ColumnarToRow (22) + : : +- CometColumnarExchange (21) + : : +- RowToColumnar (20) + : : +- * HashAggregate (19) + : : +- * 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 (42) + : +- * Project (41) + : +- Window (40) + : +- * ColumnarToRow (39) + : +- CometSort (38) + : +- CometColumnarExchange (37) + : +- RowToColumnar (36) + : +- * HashAggregate (35) + : +- * ColumnarToRow (34) + : +- ReusedExchange (33) + +- BroadcastExchange (50) + +- * Project (49) + +- Window (48) + +- * ColumnarToRow (47) + +- CometSort (46) + +- ReusedExchange (45) (1) Scan parquet spark_catalog.default.item @@ -140,142 +148,166 @@ 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] -(20) Exchange +(20) RowToColumnar 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=1] -(21) HashAggregate [codegen id : 2] +(21) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(22) ColumnarToRow [codegen id : 2] +Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] + +(23) 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))] 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] -(22) Exchange +(24) RowToColumnar +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] + +(25) CometColumnarExchange 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=2] +Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(23) Sort [codegen id : 3] +(26) CometSort 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 +Arguments: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18], [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] -(24) Window +(27) ColumnarToRow [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] + +(28) 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] -(25) Filter [codegen id : 4] +(29) 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)) -(26) Window +(30) 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] -(27) Filter [codegen id : 13] +(31) 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 : 13] +(32) 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] +(33) ReusedExchange [Reuses operator id: 21] Output [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] -(30) HashAggregate [codegen id : 6] +(34) ColumnarToRow [codegen id : 6] +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] + +(35) 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))] 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#28] -(31) Exchange +(36) RowToColumnar +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] + +(37) CometColumnarExchange 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=3] +Arguments: hashpartitioning(i_category#21, i_brand#22, cc_name#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(32) Sort [codegen id : 7] +(38) CometSort 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 +Arguments: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28], [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] -(33) Window +(39) ColumnarToRow [codegen id : 7] +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] + +(40) Window 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 : 8] +(41) 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 +(42) 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=4] -(36) BroadcastHashJoin [codegen id : 13] +(43) 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 : 13] +(44) 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] +(45) ReusedExchange [Reuses operator id: 37] Output [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] -(39) Sort [codegen id : 11] +(46) CometSort 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 +Arguments: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35], [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] -(40) Window +(47) ColumnarToRow [codegen id : 11] +Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] + +(48) Window 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 : 12] +(49) 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 +(50) 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=5] -(43) BroadcastHashJoin [codegen id : 13] +(51) 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 : 13] +(52) 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] -(45) TakeOrderedAndProject +(53) 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] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = cs_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (49) -+- * ColumnarToRow (48) - +- CometFilter (47) - +- CometScan parquet spark_catalog.default.date_dim (46) +BroadcastExchange (57) ++- * ColumnarToRow (56) + +- CometFilter (55) + +- CometScan parquet spark_catalog.default.date_dim (54) -(46) Scan parquet spark_catalog.default.date_dim +(54) 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 -(47) CometFilter +(55) 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)) -(48) ColumnarToRow [codegen id : 1] +(56) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -(49) BroadcastExchange +(57) 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=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 daeafbf92..c4518b4f1 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 @@ -13,41 +13,45 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (3) - Sort [i_category,i_brand,cc_name,d_year,d_moy] + ColumnarToRow InputAdapter - Exchange [i_category,i_brand,cc_name] #1 - 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 (1) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometFilter [i_item_sk,i_brand,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 - CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,cc_name] #1 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_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_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 + CometFilter [cc_call_center_sk,cc_name] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter BroadcastExchange #7 WholeStageCodegen (8) @@ -55,13 +59,16 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (7) - Sort [i_category,i_brand,cc_name,d_year,d_moy] + ColumnarToRow InputAdapter - 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 + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,cc_name] #8 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] + ColumnarToRow + InputAdapter + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 InputAdapter BroadcastExchange #9 WholeStageCodegen (12) @@ -69,6 +76,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (11) - Sort [i_category,i_brand,cc_name,d_year,d_moy] + ColumnarToRow InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + 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 9d660d776..b4abced27 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 @@ -1,87 +1,103 @@ == Physical Plan == -TakeOrderedAndProject (83) -+- * HashAggregate (82) - +- Exchange (81) - +- * HashAggregate (80) - +- Union (79) - :- * HashAggregate (68) - : +- Exchange (67) - : +- * HashAggregate (66) - : +- Union (65) - : :- * 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) - : +- * 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) - : +- Exchange (72) - : +- * HashAggregate (71) - : +- * HashAggregate (70) - : +- ReusedExchange (69) - +- * HashAggregate (78) - +- Exchange (77) - +- * HashAggregate (76) - +- * HashAggregate (75) - +- ReusedExchange (74) +* ColumnarToRow (99) ++- CometTakeOrderedAndProject (98) + +- CometHashAggregate (97) + +- CometColumnarExchange (96) + +- RowToColumnar (95) + +- * HashAggregate (94) + +- Union (93) + :- * HashAggregate (76) + : +- * ColumnarToRow (75) + : +- CometColumnarExchange (74) + : +- RowToColumnar (73) + : +- * HashAggregate (72) + : +- Union (71) + : :- * HashAggregate (24) + : : +- * ColumnarToRow (23) + : : +- CometColumnarExchange (22) + : : +- RowToColumnar (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 (45) + : : +- * ColumnarToRow (44) + : : +- CometColumnarExchange (43) + : : +- RowToColumnar (42) + : : +- * HashAggregate (41) + : : +- * ColumnarToRow (40) + : : +- CometProject (39) + : : +- CometBroadcastHashJoin (38) + : : :- CometProject (34) + : : : +- CometBroadcastHashJoin (33) + : : : :- CometUnion (31) + : : : : :- CometProject (27) + : : : : : +- CometFilter (26) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (25) + : : : : +- CometProject (30) + : : : : +- CometFilter (29) + : : : : +- CometScan parquet spark_catalog.default.catalog_returns (28) + : : : +- ReusedExchange (32) + : : +- CometBroadcastExchange (37) + : : +- CometFilter (36) + : : +- CometScan parquet spark_catalog.default.catalog_page (35) + : +- * HashAggregate (70) + : +- * ColumnarToRow (69) + : +- CometColumnarExchange (68) + : +- RowToColumnar (67) + : +- * HashAggregate (66) + : +- * ColumnarToRow (65) + : +- CometProject (64) + : +- CometBroadcastHashJoin (63) + : :- CometProject (59) + : : +- CometBroadcastHashJoin (58) + : : :- CometUnion (56) + : : : :- CometProject (48) + : : : : +- CometFilter (47) + : : : : +- CometScan parquet spark_catalog.default.web_sales (46) + : : : +- CometProject (55) + : : : +- CometBroadcastHashJoin (54) + : : : :- CometBroadcastExchange (50) + : : : : +- CometScan parquet spark_catalog.default.web_returns (49) + : : : +- CometProject (53) + : : : +- CometFilter (52) + : : : +- CometScan parquet spark_catalog.default.web_sales (51) + : : +- ReusedExchange (57) + : +- CometBroadcastExchange (62) + : +- CometFilter (61) + : +- CometScan parquet spark_catalog.default.web_site (60) + :- * HashAggregate (84) + : +- * ColumnarToRow (83) + : +- CometColumnarExchange (82) + : +- RowToColumnar (81) + : +- * HashAggregate (80) + : +- * HashAggregate (79) + : +- * ColumnarToRow (78) + : +- ReusedExchange (77) + +- * HashAggregate (92) + +- * ColumnarToRow (91) + +- CometColumnarExchange (90) + +- RowToColumnar (89) + +- * HashAggregate (88) + +- * HashAggregate (87) + +- * ColumnarToRow (86) + +- ReusedExchange (85) (1) Scan parquet spark_catalog.default.store_sales @@ -182,18 +198,24 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledV 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] -(21) Exchange +(21) RowToColumnar 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] -(22) HashAggregate [codegen id : 2] +(22) CometColumnarExchange +Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] +Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(23) ColumnarToRow [codegen id : 2] +Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] + +(24) 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))#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] -(23) Scan parquet spark_catalog.default.catalog_sales +(25) 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 [] @@ -201,15 +223,15 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#46), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct -(24) CometFilter +(26) 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) -(25) CometProject +(27) 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] -(26) Scan parquet spark_catalog.default.catalog_returns +(28) 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 [] @@ -217,76 +239,82 @@ PartitionFilters: [isnotnull(cr_returned_date_sk#57), dynamicpruningexpression(c PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct -(27) CometFilter +(29) 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) -(28) CometProject +(30) 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] -(29) CometUnion +(31) 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] -(30) ReusedExchange [Reuses operator id: 11] +(32) ReusedExchange [Reuses operator id: 11] Output [1]: [d_date_sk#64] -(31) CometBroadcastHashJoin +(33) 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 -(32) CometProject +(34) 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] -(33) Scan parquet spark_catalog.default.catalog_page +(35) 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 -(34) CometFilter +(36) CometFilter Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] Condition : isnotnull(cp_catalog_page_sk#65) -(35) CometBroadcastExchange +(37) 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 +(38) 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 +(39) 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] +(40) 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] +(41) 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 +(42) RowToColumnar +Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] + +(43) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] + +(44) ColumnarToRow [codegen id : 4] 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] +(45) 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 +(46) 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 [] @@ -294,250 +322,284 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#87), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(43) CometFilter +(47) 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) -(44) CometProject +(48) 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] -(45) Scan parquet spark_catalog.default.web_returns +(49) 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#99), dynamicpruningexpression(wr_returned_date_sk#99 IN dynamicpruning#88)] ReadSchema: struct -(46) CometBroadcastExchange +(50) 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] -(47) Scan parquet spark_catalog.default.web_sales +(51) 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 -(48) CometFilter +(52) 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) CometProject +(53) 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) CometBroadcastHashJoin +(54) 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) CometProject +(55) 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) CometUnion +(56) 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: 11] +(57) ReusedExchange [Reuses operator id: 11] Output [1]: [d_date_sk#110] -(54) CometBroadcastHashJoin +(58) 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) CometProject +(59) 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 +(60) Scan parquet spark_catalog.default.web_site 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 +(61) CometFilter Input [2]: [web_site_sk#111, web_site_id#112] Condition : isnotnull(web_site_sk#111) -(58) CometBroadcastExchange +(62) CometBroadcastExchange Input [2]: [web_site_sk#111, web_site_id#112] Arguments: [web_site_sk#111, web_site_id#112] -(59) CometBroadcastHashJoin +(63) 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) CometProject +(64) 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) ColumnarToRow [codegen id : 5] +(65) 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 : 5] +(66) 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 +(67) RowToColumnar 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 : 6] +(68) CometColumnarExchange +Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] +Arguments: hashpartitioning(web_site_id#112, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(69) ColumnarToRow [codegen id : 6] +Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] + +(70) 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 +(71) Union -(66) HashAggregate [codegen id : 7] +(72) 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 +(73) RowToColumnar +Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] + +(74) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=4] + +(75) ColumnarToRow [codegen id : 8] 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 : 8] +(76) 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] +(77) ReusedExchange [Reuses operator id: 74] Output [8]: [channel#148, id#149, sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155] -(70) HashAggregate [codegen id : 16] +(78) ColumnarToRow [codegen id : 16] +Input [8]: [channel#148, id#149, sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155] + +(79) 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] +(80) 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 +(81) RowToColumnar +Input [7]: [channel#148, sum#168, isEmpty#169, sum#170, isEmpty#171, sum#172, isEmpty#173] + +(82) CometColumnarExchange 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] +Arguments: hashpartitioning(channel#148, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(73) HashAggregate [codegen id : 17] +(83) ColumnarToRow [codegen id : 17] +Input [7]: [channel#148, sum#168, isEmpty#169, sum#170, isEmpty#171, sum#172, isEmpty#173] + +(84) 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] +(85) ReusedExchange [Reuses operator id: 74] Output [8]: [channel#181, id#182, sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188] -(75) HashAggregate [codegen id : 25] +(86) ColumnarToRow [codegen id : 25] +Input [8]: [channel#181, id#182, sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188] + +(87) 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 : 25] +(88) HashAggregate [codegen id : 25] Input [3]: [sales#192, returns#193, profit#194] Keys: [] 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 +(89) RowToColumnar 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 : 26] +(90) CometColumnarExchange +Input [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(91) ColumnarToRow [codegen id : 26] +Input [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] + +(92) HashAggregate [codegen id : 26] Input [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] Keys: [] 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 +(93) Union -(80) HashAggregate [codegen id : 27] +(94) 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#38, id#39, sales#145, returns#146, profit#147] -(81) Exchange +(95) RowToColumnar +Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] + +(96) CometColumnarExchange 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] +Arguments: hashpartitioning(channel#38, id#39, sales#145, returns#146, profit#147, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(82) HashAggregate [codegen id : 28] +(97) CometHashAggregate 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#38, id#39, sales#145, returns#146, profit#147] -(83) TakeOrderedAndProject +(98) CometTakeOrderedAndProject +Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#38 ASC NULLS FIRST,id#39 ASC NULLS FIRST], output=[channel#38,id#39,sales#145,returns#146,profit#147]), [channel#38, id#39, sales#145, returns#146, profit#147], 100, [channel#38 ASC NULLS FIRST, id#39 ASC NULLS FIRST], [channel#38, id#39, sales#145, returns#146, profit#147] + +(99) ColumnarToRow [codegen id : 28] 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 ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (88) -+- * ColumnarToRow (87) - +- CometProject (86) - +- CometFilter (85) - +- CometScan parquet spark_catalog.default.date_dim (84) +BroadcastExchange (104) ++- * ColumnarToRow (103) + +- CometProject (102) + +- CometFilter (101) + +- CometScan parquet spark_catalog.default.date_dim (100) -(84) Scan parquet spark_catalog.default.date_dim +(100) 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 -(85) CometFilter +(101) 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)) -(86) CometProject +(102) CometProject Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(87) ColumnarToRow [codegen id : 1] +(103) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(88) BroadcastExchange +(104) BroadcastExchange Input [1]: [d_date_sk#22] 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 = 23 Hosting Expression = cs_sold_date_sk#46 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 25 Hosting Expression = cs_sold_date_sk#46 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#57 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 28 Hosting Expression = cr_returned_date_sk#57 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#87 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#87 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 45 Hosting Expression = wr_returned_date_sk#99 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 49 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 c3b00538b..733975649 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,124 +1,140 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (28) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Exchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (27) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - 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 (7) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - 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] +WholeStageCodegen (28) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [channel,id,sales,returns,profit] + CometColumnarExchange [channel,id,sales,returns,profit] #1 + RowToColumnar + WholeStageCodegen (27) + HashAggregate [channel,id,sales,returns,profit] + InputAdapter + Union + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel,id] #2 + RowToColumnar + WholeStageCodegen (7) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange [s_store_id] #3 - WholeStageCodegen (1) - HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Union + 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] ColumnarToRow InputAdapter - CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] - CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - 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,ss_ext_sales_price,ss_net_profit,ss_sold_date_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 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_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 - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #6 - CometFilter [s_store_sk,s_store_id] - 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] #7 - WholeStageCodegen (3) - HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + CometColumnarExchange [s_store_id] #3 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + ColumnarToRow + InputAdapter + CometProject [sales_price,profit,return_amt,net_loss,s_store_id] + CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] + CometProject [store_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + 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,ss_ext_sales_price,ss_net_profit,ss_sold_date_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 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_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 + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [s_store_sk,s_store_id] #6 + CometFilter [s_store_sk,s_store_id] + 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] ColumnarToRow InputAdapter - CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - 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,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_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 - ReusedExchange [d_date_sk] #5 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #8 - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - 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] #9 - WholeStageCodegen (5) - HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + CometColumnarExchange [cp_catalog_page_id] #7 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + ColumnarToRow + InputAdapter + CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [page_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + 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,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_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 + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #8 + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + 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] ColumnarToRow InputAdapter - CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] - CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - 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,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - 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,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] - CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #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_web_site_sk,ws_order_number,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - ReusedExchange [d_date_sk] #5 - CometBroadcastExchange [web_site_sk,web_site_id] #11 - CometFilter [web_site_sk,web_site_id] - 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] #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 (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 #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 - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometColumnarExchange [web_site_id] #9 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + ColumnarToRow + InputAdapter + CometProject [sales_price,profit,return_amt,net_loss,web_site_id] + CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] + CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + 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,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + 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,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] + CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #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_web_site_sk,ws_order_number,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange [web_site_sk,web_site_id] #11 + CometFilter [web_site_sk,web_site_id] + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel] #12 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange #13 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 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 fbf8c8ead..4e037389f 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,44 +1,48 @@ == Physical Plan == -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) +TakeOrderedAndProject (44) ++- * Filter (43) + +- * HashAggregate (42) + +- * ColumnarToRow (41) + +- CometColumnarExchange (40) + +- RowToColumnar (39) + +- * HashAggregate (38) + +- * Project (37) + +- * BroadcastHashJoin Inner BuildRight (36) + :- * 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 (35) + +- * Project (34) + +- * BroadcastHashJoin Inner BuildRight (33) + :- * ColumnarToRow (22) + : +- CometFilter (21) + : +- CometScan parquet spark_catalog.default.item (20) + +- BroadcastExchange (32) + +- * Filter (31) + +- * HashAggregate (30) + +- * ColumnarToRow (29) + +- CometColumnarExchange (28) + +- RowToColumnar (27) + +- * HashAggregate (26) + +- * ColumnarToRow (25) + +- CometFilter (24) + +- CometScan parquet spark_catalog.default.item (23) (1) Scan parquet spark_catalog.default.customer_address @@ -167,152 +171,162 @@ 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 +(27) RowToColumnar 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] +(28) CometColumnarExchange +Input [3]: [i_category#17, sum#20, count#21] +Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(29) ColumnarToRow [codegen id : 2] +Input [3]: [i_category#17, sum#20, count#21] + +(30) 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] +(31) Filter [codegen id : 2] Input [2]: [avg(i_current_price)#23, i_category#17] Condition : isnotnull(avg(i_current_price)#23) -(30) BroadcastExchange +(32) 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] +(33) 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#14 as decimal(14,7)) > (1.2 * avg(i_current_price)#23)) -(32) Project [codegen id : 3] +(34) 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] -(33) BroadcastExchange +(35) BroadcastExchange Input [1]: [i_item_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(34) BroadcastHashJoin [codegen id : 4] +(36) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_item_sk#5] Right keys [1]: [i_item_sk#13] Join type: Inner Join condition: None -(35) Project [codegen id : 4] +(37) Project [codegen id : 4] Output [1]: [ca_state#2] Input [3]: [ca_state#2, ss_item_sk#5, i_item_sk#13] -(36) HashAggregate [codegen id : 4] +(38) HashAggregate [codegen id : 4] Input [1]: [ca_state#2] Keys [1]: [ca_state#2] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#24] Results [2]: [ca_state#2, count#25] -(37) Exchange +(39) RowToColumnar +Input [2]: [ca_state#2, count#25] + +(40) CometColumnarExchange Input [2]: [ca_state#2, count#25] -Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(38) HashAggregate [codegen id : 5] +(41) ColumnarToRow [codegen id : 5] +Input [2]: [ca_state#2, count#25] + +(42) 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)#26] Results [2]: [ca_state#2 AS state#27, count(1)#26 AS cnt#28] -(39) Filter [codegen id : 5] +(43) Filter [codegen id : 5] Input [2]: [state#27, cnt#28] Condition : (cnt#28 >= 10) -(40) TakeOrderedAndProject +(44) 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 = 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) +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan parquet spark_catalog.default.date_dim (45) -(41) Scan parquet spark_catalog.default.date_dim +(45) 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 -(42) CometFilter +(46) 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)) -(43) CometProject +(47) CometProject Input [2]: [d_date_sk#9, d_month_seq#10] Arguments: [d_date_sk#9], [d_date_sk#9] -(44) ColumnarToRow [codegen id : 1] +(48) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(45) BroadcastExchange +(49) BroadcastExchange Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -Subquery:2 Hosting operator id = 42 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] +Subquery:2 Hosting operator id = 46 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] -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) +Subquery:3 Hosting operator id = 45 Hosting Expression = Subquery scalar-subquery#11, [id=#12] +* ColumnarToRow (56) ++- CometHashAggregate (55) + +- CometColumnarExchange (54) + +- CometHashAggregate (53) + +- CometProject (52) + +- CometFilter (51) + +- CometScan parquet spark_catalog.default.date_dim (50) -(46) Scan parquet spark_catalog.default.date_dim +(50) 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 -(47) CometFilter +(51) 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)) -(48) CometProject +(52) CometProject Input [3]: [d_month_seq#29, d_year#30, d_moy#31] Arguments: [d_month_seq#29], [d_month_seq#29] -(49) CometHashAggregate +(53) CometHashAggregate Input [1]: [d_month_seq#29] Keys [1]: [d_month_seq#29] Functions: [] -(50) ColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#29] - -(51) Exchange +(54) CometColumnarExchange Input [1]: [d_month_seq#29] -Arguments: hashpartitioning(d_month_seq#29, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Arguments: hashpartitioning(d_month_seq#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(52) HashAggregate [codegen id : 2] +(55) CometHashAggregate Input [1]: [d_month_seq#29] Keys [1]: [d_month_seq#29] Functions: [] -Aggregate Attributes: [] -Results [1]: [d_month_seq#29] + +(56) ColumnarToRow [codegen id : 1] +Input [1]: [d_month_seq#29] Subquery:4 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-spark4_0/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/simplified.txt index febbfd32f..e28908700 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 @@ -2,74 +2,76 @@ TakeOrderedAndProject [cnt,state] WholeStageCodegen (5) Filter [cnt] HashAggregate [ca_state,count] [count(1),state,cnt,count] - InputAdapter - Exchange [ca_state] #1 - WholeStageCodegen (4) - HashAggregate [ca_state] [count,count] - Project [ca_state] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometProject [ca_state,ss_item_sk] - CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] - CometProject [ca_state,ss_item_sk,ss_sold_date_sk] - CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] - CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_customer_sk,ss_sold_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 #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - 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 + ColumnarToRow + InputAdapter + CometColumnarExchange [ca_state] #1 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [ca_state] [count,count] + Project [ca_state] + BroadcastHashJoin [ss_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometProject [ca_state,ss_item_sk] + CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] + CometProject [ca_state,ss_item_sk,ss_sold_date_sk] + CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] + CometProject [ca_state,c_customer_sk] + CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_customer_sk,ss_sold_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 #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + ReusedSubquery [d_month_seq] #2 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + Subquery #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [d_month_seq] + CometColumnarExchange [d_month_seq] #5 CometHashAggregate [d_month_seq] CometProject [d_month_seq] CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - 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 #7 - WholeStageCodegen (3) - Project [i_item_sk] - BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_current_price,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - InputAdapter - 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] #9 - WholeStageCodegen (1) - HashAggregate [i_category,i_current_price] [sum,count,sum,count] - ColumnarToRow - InputAdapter - CometFilter [i_current_price,i_category] - CometScan parquet spark_catalog.default.item [i_current_price,i_category] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + 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 #7 + WholeStageCodegen (3) + Project [i_item_sk] + BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_current_price,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] + InputAdapter + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category] #9 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [i_category,i_current_price] [sum,count,sum,count] + ColumnarToRow + InputAdapter + CometFilter [i_current_price,i_category] + CometScan parquet spark_catalog.default.item [i_current_price,i_category] 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 b74018441..30c42999c 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 @@ -1,185 +1,193 @@ == Physical Plan == -* Sort (181) -+- Exchange (180) - +- * Project (179) - +- * SortMergeJoin Inner (178) - :- * Sort (110) - : +- Exchange (109) - : +- * HashAggregate (108) - : +- * HashAggregate (107) - : +- * Project (106) - : +- * BroadcastHashJoin Inner BuildRight (105) - : :- * Project (99) - : : +- * BroadcastHashJoin Inner BuildRight (98) - : : :- * Project (96) - : : : +- * BroadcastHashJoin Inner BuildRight (95) - : : : :- * Project (90) - : : : : +- * BroadcastHashJoin Inner BuildRight (89) - : : : : :- * Project (87) - : : : : : +- * BroadcastHashJoin Inner BuildRight (86) - : : : : : :- * Project (81) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (80) - : : : : : : :- * Project (78) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (77) - : : : : : : : :- * Project (72) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (71) - : : : : : : : : :- * Project (66) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (65) - : : : : : : : : : :- * Project (63) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (62) - : : : : : : : : : : :- * Project (57) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (56) - : : : : : : : : : : : :- * Project (54) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (53) - : : : : : : : : : : : : :- * Project (48) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (47) - : : : : : : : : : : : : : :- * Project (42) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : : : : : : : : : : : :- * Project (36) - : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (35) - : : : : : : : : : : : : : : : :- * Project (33) - : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (32) - : : : : : : : : : : : : : : : : :- * Sort (11) - : : : : : : : : : : : : : : : : : +- Exchange (10) - : : : : : : : : : : : : : : : : : +- * ColumnarToRow (9) - : : : : : : : : : : : : : : : : : +- CometProject (8) - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) - : : : : : : : : : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- CometProject (6) - : : : : : : : : : : : : : : : : : +- CometFilter (5) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : : : : : : : : : : : : : : : +- * Sort (31) - : : : : : : : : : : : : : : : : +- * Project (30) - : : : : : : : : : : : : : : : : +- * Filter (29) - : : : : : : : : : : : : : : : : +- * HashAggregate (28) - : : : : : : : : : : : : : : : : +- Exchange (27) - : : : : : : : : : : : : : : : : +- * HashAggregate (26) - : : : : : : : : : : : : : : : : +- * Project (25) - : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (24) - : : : : : : : : : : : : : : : : :- * Sort (17) - : : : : : : : : : : : : : : : : : +- Exchange (16) - : : : : : : : : : : : : : : : : : +- * ColumnarToRow (15) - : : : : : : : : : : : : : : : : : +- CometProject (14) - : : : : : : : : : : : : : : : : : +- CometFilter (13) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (12) - : : : : : : : : : : : : : : : : +- * Sort (23) - : : : : : : : : : : : : : : : : +- Exchange (22) - : : : : : : : : : : : : : : : : +- * ColumnarToRow (21) - : : : : : : : : : : : : : : : : +- CometProject (20) - : : : : : : : : : : : : : : : : +- CometFilter (19) - : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (18) - : : : : : : : : : : : : : : : +- ReusedExchange (34) - : : : : : : : : : : : : : : +- BroadcastExchange (40) - : : : : : : : : : : : : : : +- * ColumnarToRow (39) - : : : : : : : : : : : : : : +- CometFilter (38) - : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (37) - : : : : : : : : : : : : : +- BroadcastExchange (46) - : : : : : : : : : : : : : +- * ColumnarToRow (45) - : : : : : : : : : : : : : +- CometFilter (44) - : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (43) - : : : : : : : : : : : : +- BroadcastExchange (52) - : : : : : : : : : : : : +- * ColumnarToRow (51) - : : : : : : : : : : : : +- CometFilter (50) - : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (49) - : : : : : : : : : : : +- ReusedExchange (55) - : : : : : : : : : : +- BroadcastExchange (61) - : : : : : : : : : : +- * ColumnarToRow (60) - : : : : : : : : : : +- CometFilter (59) - : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (58) - : : : : : : : : : +- ReusedExchange (64) - : : : : : : : : +- BroadcastExchange (70) - : : : : : : : : +- * ColumnarToRow (69) - : : : : : : : : +- CometFilter (68) - : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (67) - : : : : : : : +- BroadcastExchange (76) - : : : : : : : +- * ColumnarToRow (75) - : : : : : : : +- CometFilter (74) - : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (73) - : : : : : : +- ReusedExchange (79) - : : : : : +- BroadcastExchange (85) - : : : : : +- * ColumnarToRow (84) - : : : : : +- CometFilter (83) - : : : : : +- CometScan parquet spark_catalog.default.customer_address (82) - : : : : +- ReusedExchange (88) - : : : +- BroadcastExchange (94) - : : : +- * ColumnarToRow (93) - : : : +- CometFilter (92) - : : : +- CometScan parquet spark_catalog.default.income_band (91) - : : +- ReusedExchange (97) - : +- BroadcastExchange (104) - : +- * ColumnarToRow (103) - : +- CometProject (102) - : +- CometFilter (101) - : +- CometScan parquet spark_catalog.default.item (100) - +- * Sort (177) - +- Exchange (176) - +- * HashAggregate (175) - +- * HashAggregate (174) - +- * Project (173) - +- * BroadcastHashJoin Inner BuildRight (172) - :- * Project (170) - : +- * BroadcastHashJoin Inner BuildRight (169) - : :- * Project (167) - : : +- * BroadcastHashJoin Inner BuildRight (166) - : : :- * Project (164) - : : : +- * BroadcastHashJoin Inner BuildRight (163) - : : : :- * Project (161) - : : : : +- * BroadcastHashJoin Inner BuildRight (160) - : : : : :- * Project (158) - : : : : : +- * BroadcastHashJoin Inner BuildRight (157) - : : : : : :- * Project (155) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (154) - : : : : : : :- * Project (152) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (151) - : : : : : : : :- * Project (149) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (148) - : : : : : : : : :- * Project (146) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (145) - : : : : : : : : : :- * Project (143) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (142) - : : : : : : : : : : :- * Project (140) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (139) - : : : : : : : : : : : :- * Project (137) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (136) - : : : : : : : : : : : : :- * Project (134) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (133) - : : : : : : : : : : : : : :- * Project (131) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (130) - : : : : : : : : : : : : : : :- * Project (128) - : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (127) - : : : : : : : : : : : : : : : :- * Sort (121) - : : : : : : : : : : : : : : : : +- Exchange (120) - : : : : : : : : : : : : : : : : +- * ColumnarToRow (119) - : : : : : : : : : : : : : : : : +- CometProject (118) - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (117) - : : : : : : : : : : : : : : : : :- CometBroadcastExchange (113) - : : : : : : : : : : : : : : : : : +- CometFilter (112) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (111) - : : : : : : : : : : : : : : : : +- CometProject (116) - : : : : : : : : : : : : : : : : +- CometFilter (115) - : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (114) - : : : : : : : : : : : : : : : +- * Sort (126) - : : : : : : : : : : : : : : : +- * Project (125) - : : : : : : : : : : : : : : : +- * Filter (124) - : : : : : : : : : : : : : : : +- * HashAggregate (123) - : : : : : : : : : : : : : : : +- ReusedExchange (122) - : : : : : : : : : : : : : : +- ReusedExchange (129) - : : : : : : : : : : : : : +- ReusedExchange (132) - : : : : : : : : : : : : +- ReusedExchange (135) - : : : : : : : : : : : +- ReusedExchange (138) - : : : : : : : : : : +- ReusedExchange (141) - : : : : : : : : : +- ReusedExchange (144) - : : : : : : : : +- ReusedExchange (147) - : : : : : : : +- ReusedExchange (150) - : : : : : : +- ReusedExchange (153) - : : : : : +- ReusedExchange (156) - : : : : +- ReusedExchange (159) - : : : +- ReusedExchange (162) - : : +- ReusedExchange (165) - : +- ReusedExchange (168) - +- ReusedExchange (171) +* ColumnarToRow (189) ++- CometSort (188) + +- CometColumnarExchange (187) + +- RowToColumnar (186) + +- * Project (185) + +- * SortMergeJoin Inner (184) + :- * ColumnarToRow (113) + : +- CometSort (112) + : +- CometColumnarExchange (111) + : +- RowToColumnar (110) + : +- * HashAggregate (109) + : +- * HashAggregate (108) + : +- * Project (107) + : +- * BroadcastHashJoin Inner BuildRight (106) + : :- * Project (100) + : : +- * BroadcastHashJoin Inner BuildRight (99) + : : :- * Project (97) + : : : +- * BroadcastHashJoin Inner BuildRight (96) + : : : :- * Project (91) + : : : : +- * BroadcastHashJoin Inner BuildRight (90) + : : : : :- * Project (88) + : : : : : +- * BroadcastHashJoin Inner BuildRight (87) + : : : : : :- * Project (82) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (81) + : : : : : : :- * Project (79) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (78) + : : : : : : : :- * Project (73) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (72) + : : : : : : : : :- * Project (67) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (66) + : : : : : : : : : :- * Project (64) + : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (63) + : : : : : : : : : : :- * Project (58) + : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (57) + : : : : : : : : : : : :- * Project (55) + : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : : : : : : : : : : :- * Project (49) + : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (48) + : : : : : : : : : : : : : :- * Project (43) + : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (42) + : : : : : : : : : : : : : : :- * Project (37) + : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (36) + : : : : : : : : : : : : : : : :- * Project (34) + : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (33) + : : : : : : : : : : : : : : : : :- * ColumnarToRow (11) + : : : : : : : : : : : : : : : : : +- CometSort (10) + : : : : : : : : : : : : : : : : : +- CometColumnarExchange (9) + : : : : : : : : : : : : : : : : : +- CometProject (8) + : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) + : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) + : : : : : : : : : : : : : : : : : : +- CometFilter (2) + : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : : : : : : : : : : : : +- CometProject (6) + : : : : : : : : : : : : : : : : : +- CometFilter (5) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : : : : : : : : : : : : : : : +- * Sort (32) + : : : : : : : : : : : : : : : : +- * Project (31) + : : : : : : : : : : : : : : : : +- * Filter (30) + : : : : : : : : : : : : : : : : +- * HashAggregate (29) + : : : : : : : : : : : : : : : : +- * ColumnarToRow (28) + : : : : : : : : : : : : : : : : +- CometColumnarExchange (27) + : : : : : : : : : : : : : : : : +- RowToColumnar (26) + : : : : : : : : : : : : : : : : +- * HashAggregate (25) + : : : : : : : : : : : : : : : : +- * ColumnarToRow (24) + : : : : : : : : : : : : : : : : +- CometProject (23) + : : : : : : : : : : : : : : : : +- CometSortMergeJoin (22) + : : : : : : : : : : : : : : : : :- CometSort (16) + : : : : : : : : : : : : : : : : : +- CometColumnarExchange (15) + : : : : : : : : : : : : : : : : : +- CometProject (14) + : : : : : : : : : : : : : : : : : +- CometFilter (13) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (12) + : : : : : : : : : : : : : : : : +- CometSort (21) + : : : : : : : : : : : : : : : : +- CometColumnarExchange (20) + : : : : : : : : : : : : : : : : +- CometProject (19) + : : : : : : : : : : : : : : : : +- CometFilter (18) + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (17) + : : : : : : : : : : : : : : : +- ReusedExchange (35) + : : : : : : : : : : : : : : +- BroadcastExchange (41) + : : : : : : : : : : : : : : +- * ColumnarToRow (40) + : : : : : : : : : : : : : : +- CometFilter (39) + : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (38) + : : : : : : : : : : : : : +- BroadcastExchange (47) + : : : : : : : : : : : : : +- * ColumnarToRow (46) + : : : : : : : : : : : : : +- CometFilter (45) + : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (44) + : : : : : : : : : : : : +- BroadcastExchange (53) + : : : : : : : : : : : : +- * ColumnarToRow (52) + : : : : : : : : : : : : +- CometFilter (51) + : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (50) + : : : : : : : : : : : +- ReusedExchange (56) + : : : : : : : : : : +- BroadcastExchange (62) + : : : : : : : : : : +- * ColumnarToRow (61) + : : : : : : : : : : +- CometFilter (60) + : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (59) + : : : : : : : : : +- ReusedExchange (65) + : : : : : : : : +- BroadcastExchange (71) + : : : : : : : : +- * ColumnarToRow (70) + : : : : : : : : +- CometFilter (69) + : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (68) + : : : : : : : +- BroadcastExchange (77) + : : : : : : : +- * ColumnarToRow (76) + : : : : : : : +- CometFilter (75) + : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (74) + : : : : : : +- ReusedExchange (80) + : : : : : +- BroadcastExchange (86) + : : : : : +- * ColumnarToRow (85) + : : : : : +- CometFilter (84) + : : : : : +- CometScan parquet spark_catalog.default.customer_address (83) + : : : : +- ReusedExchange (89) + : : : +- BroadcastExchange (95) + : : : +- * ColumnarToRow (94) + : : : +- CometFilter (93) + : : : +- CometScan parquet spark_catalog.default.income_band (92) + : : +- ReusedExchange (98) + : +- BroadcastExchange (105) + : +- * ColumnarToRow (104) + : +- CometProject (103) + : +- CometFilter (102) + : +- CometScan parquet spark_catalog.default.item (101) + +- * ColumnarToRow (183) + +- CometSort (182) + +- CometColumnarExchange (181) + +- RowToColumnar (180) + +- * HashAggregate (179) + +- * HashAggregate (178) + +- * Project (177) + +- * BroadcastHashJoin Inner BuildRight (176) + :- * Project (174) + : +- * BroadcastHashJoin Inner BuildRight (173) + : :- * Project (171) + : : +- * BroadcastHashJoin Inner BuildRight (170) + : : :- * Project (168) + : : : +- * BroadcastHashJoin Inner BuildRight (167) + : : : :- * Project (165) + : : : : +- * BroadcastHashJoin Inner BuildRight (164) + : : : : :- * Project (162) + : : : : : +- * BroadcastHashJoin Inner BuildRight (161) + : : : : : :- * Project (159) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (158) + : : : : : : :- * Project (156) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (155) + : : : : : : : :- * Project (153) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (152) + : : : : : : : : :- * Project (150) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (149) + : : : : : : : : : :- * Project (147) + : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (146) + : : : : : : : : : : :- * Project (144) + : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (143) + : : : : : : : : : : : :- * Project (141) + : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (140) + : : : : : : : : : : : : :- * Project (138) + : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (137) + : : : : : : : : : : : : : :- * Project (135) + : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (134) + : : : : : : : : : : : : : : :- * Project (132) + : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (131) + : : : : : : : : : : : : : : : :- * ColumnarToRow (124) + : : : : : : : : : : : : : : : : +- CometSort (123) + : : : : : : : : : : : : : : : : +- CometColumnarExchange (122) + : : : : : : : : : : : : : : : : +- CometProject (121) + : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (120) + : : : : : : : : : : : : : : : : :- CometBroadcastExchange (116) + : : : : : : : : : : : : : : : : : +- CometFilter (115) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (114) + : : : : : : : : : : : : : : : : +- CometProject (119) + : : : : : : : : : : : : : : : : +- CometFilter (118) + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (117) + : : : : : : : : : : : : : : : +- * Sort (130) + : : : : : : : : : : : : : : : +- * Project (129) + : : : : : : : : : : : : : : : +- * Filter (128) + : : : : : : : : : : : : : : : +- * HashAggregate (127) + : : : : : : : : : : : : : : : +- * ColumnarToRow (126) + : : : : : : : : : : : : : : : +- ReusedExchange (125) + : : : : : : : : : : : : : : +- ReusedExchange (133) + : : : : : : : : : : : : : +- ReusedExchange (136) + : : : : : : : : : : : : +- ReusedExchange (139) + : : : : : : : : : : : +- ReusedExchange (142) + : : : : : : : : : : +- ReusedExchange (145) + : : : : : : : : : +- ReusedExchange (148) + : : : : : : : : +- ReusedExchange (151) + : : : : : : : +- ReusedExchange (154) + : : : : : : +- ReusedExchange (157) + : : : : : +- ReusedExchange (160) + : : : : +- ReusedExchange (163) + : : : +- ReusedExchange (166) + : : +- ReusedExchange (169) + : +- ReusedExchange (172) + +- ReusedExchange (175) (1) Scan parquet spark_catalog.default.store_sales @@ -222,16 +230,16 @@ Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number# 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] Arguments: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -(9) ColumnarToRow [codegen id : 1] +(9) CometColumnarExchange Input [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(10) Exchange +(10) CometSort Input [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1 ASC NULLS FIRST] -(11) Sort [codegen id : 2] +(11) ColumnarToRow [codegen id : 1] Input [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 (12) Scan parquet spark_catalog.default.catalog_sales Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] @@ -248,450 +256,458 @@ Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -(15) ColumnarToRow [codegen id : 3] +(15) CometColumnarExchange Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] +Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) Exchange +(16) CometSort Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST] -(17) Sort [codegen id : 4] -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST], false, 0 - -(18) Scan parquet spark_catalog.default.catalog_returns +(17) Scan parquet spark_catalog.default.catalog_returns Output [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(19) CometFilter +(18) CometFilter Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) -(20) CometProject +(19) CometProject Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -(21) ColumnarToRow [codegen id : 5] +(20) CometColumnarExchange Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) Exchange +(21) CometSort Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST] -(23) Sort [codegen id : 6] -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST], false, 0 +(22) CometSortMergeJoin +Left output [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] +Right output [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number#22], Inner -(24) SortMergeJoin [codegen id : 7] -Left keys [2]: [cs_item_sk#17, cs_order_number#18] -Right keys [2]: [cr_item_sk#21, cr_order_number#22] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 7] -Output [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +(23) CometProject Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +Arguments: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] + +(24) ColumnarToRow [codegen id : 2] +Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -(26) HashAggregate [codegen id : 7] +(25) HashAggregate [codegen id : 2] Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Keys [1]: [cs_item_sk#17] Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] Aggregate Attributes [3]: [sum#27, sum#28, isEmpty#29] Results [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] -(27) Exchange +(26) RowToColumnar +Input [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] + +(27) CometColumnarExchange Input [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(28) HashAggregate [codegen id : 8] +(28) ColumnarToRow [codegen id : 3] +Input [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] + +(29) HashAggregate [codegen id : 3] Input [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] Keys [1]: [cs_item_sk#17] Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#19))#33, sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))#34] Results [3]: [cs_item_sk#17, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#19))#33,17,2) AS sale#35, sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))#34 AS refund#36] -(29) Filter [codegen id : 8] +(30) Filter [codegen id : 3] Input [3]: [cs_item_sk#17, sale#35, refund#36] Condition : ((isnotnull(sale#35) AND isnotnull(refund#36)) AND (cast(sale#35 as decimal(21,2)) > (2 * refund#36))) -(30) Project [codegen id : 8] +(31) Project [codegen id : 3] Output [1]: [cs_item_sk#17] Input [3]: [cs_item_sk#17, sale#35, refund#36] -(31) Sort [codegen id : 8] +(32) Sort [codegen id : 3] Input [1]: [cs_item_sk#17] Arguments: [cs_item_sk#17 ASC NULLS FIRST], false, 0 -(32) SortMergeJoin [codegen id : 24] +(33) SortMergeJoin [codegen id : 19] Left keys [1]: [ss_item_sk#1] Right keys [1]: [cs_item_sk#17] Join type: Inner Join condition: None -(33) Project [codegen id : 24] +(34) Project [codegen id : 19] Output [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] Input [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#17] -(34) ReusedExchange [Reuses operator id: 185] +(35) ReusedExchange [Reuses operator id: 193] Output [2]: [d_date_sk#37, d_year#38] -(35) BroadcastHashJoin [codegen id : 24] +(36) BroadcastHashJoin [codegen id : 19] Left keys [1]: [ss_sold_date_sk#12] Right keys [1]: [d_date_sk#37] Join type: Inner Join condition: None -(36) Project [codegen id : 24] +(37) Project [codegen id : 19] Output [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38] Input [13]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#37, d_year#38] -(37) Scan parquet spark_catalog.default.store +(38) Scan parquet spark_catalog.default.store Output [3]: [s_store_sk#39, s_store_name#40, s_zip#41] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_zip)] ReadSchema: struct -(38) CometFilter +(39) CometFilter Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] Condition : ((isnotnull(s_store_sk#39) AND isnotnull(s_store_name#40)) AND isnotnull(s_zip#41)) -(39) ColumnarToRow [codegen id : 10] +(40) ColumnarToRow [codegen id : 5] Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] -(40) BroadcastExchange +(41) BroadcastExchange Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(41) BroadcastHashJoin [codegen id : 24] +(42) BroadcastHashJoin [codegen id : 19] Left keys [1]: [ss_store_sk#6] Right keys [1]: [s_store_sk#39] Join type: Inner Join condition: None -(42) Project [codegen id : 24] +(43) Project [codegen id : 19] Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41] 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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_sk#39, s_store_name#40, s_zip#41] -(43) Scan parquet spark_catalog.default.customer +(44) Scan parquet spark_catalog.default.customer Output [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(44) CometFilter +(45) CometFilter Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] Condition : (((((isnotnull(c_customer_sk#42) AND isnotnull(c_first_sales_date_sk#47)) AND isnotnull(c_first_shipto_date_sk#46)) AND isnotnull(c_current_cdemo_sk#43)) AND isnotnull(c_current_hdemo_sk#44)) AND isnotnull(c_current_addr_sk#45)) -(45) ColumnarToRow [codegen id : 11] +(46) ColumnarToRow [codegen id : 6] Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] -(46) BroadcastExchange +(47) BroadcastExchange Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(47) BroadcastHashJoin [codegen id : 24] +(48) BroadcastHashJoin [codegen id : 19] Left keys [1]: [ss_customer_sk#2] Right keys [1]: [c_customer_sk#42] Join type: Inner Join condition: None -(48) Project [codegen id : 24] +(49) Project [codegen id : 19] Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] -(49) Scan parquet spark_catalog.default.date_dim +(50) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#48, d_year#49] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct -(50) CometFilter +(51) CometFilter Input [2]: [d_date_sk#48, d_year#49] Condition : isnotnull(d_date_sk#48) -(51) ColumnarToRow [codegen id : 12] +(52) ColumnarToRow [codegen id : 7] Input [2]: [d_date_sk#48, d_year#49] -(52) BroadcastExchange +(53) BroadcastExchange Input [2]: [d_date_sk#48, d_year#49] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(53) BroadcastHashJoin [codegen id : 24] +(54) BroadcastHashJoin [codegen id : 19] Left keys [1]: [c_first_sales_date_sk#47] Right keys [1]: [d_date_sk#48] Join type: Inner Join condition: None -(54) Project [codegen id : 24] +(55) Project [codegen id : 19] Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, d_year#49] Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47, d_date_sk#48, d_year#49] -(55) ReusedExchange [Reuses operator id: 52] +(56) ReusedExchange [Reuses operator id: 53] Output [2]: [d_date_sk#50, d_year#51] -(56) BroadcastHashJoin [codegen id : 24] +(57) BroadcastHashJoin [codegen id : 19] Left keys [1]: [c_first_shipto_date_sk#46] Right keys [1]: [d_date_sk#50] Join type: Inner Join condition: None -(57) Project [codegen id : 24] +(58) Project [codegen id : 19] Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, d_year#49, d_date_sk#50, d_year#51] -(58) Scan parquet spark_catalog.default.customer_demographics +(59) Scan parquet spark_catalog.default.customer_demographics Output [2]: [cd_demo_sk#52, cd_marital_status#53] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status)] ReadSchema: struct -(59) CometFilter +(60) CometFilter Input [2]: [cd_demo_sk#52, cd_marital_status#53] Condition : (isnotnull(cd_demo_sk#52) AND isnotnull(cd_marital_status#53)) -(60) ColumnarToRow [codegen id : 14] +(61) ColumnarToRow [codegen id : 9] Input [2]: [cd_demo_sk#52, cd_marital_status#53] -(61) BroadcastExchange +(62) BroadcastExchange Input [2]: [cd_demo_sk#52, cd_marital_status#53] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(62) BroadcastHashJoin [codegen id : 24] +(63) BroadcastHashJoin [codegen id : 19] Left keys [1]: [ss_cdemo_sk#3] Right keys [1]: [cd_demo_sk#52] Join type: Inner Join condition: None -(63) Project [codegen id : 24] +(64) Project [codegen id : 19] Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_marital_status#53] Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_demo_sk#52, cd_marital_status#53] -(64) ReusedExchange [Reuses operator id: 61] +(65) ReusedExchange [Reuses operator id: 62] Output [2]: [cd_demo_sk#54, cd_marital_status#55] -(65) BroadcastHashJoin [codegen id : 24] +(66) BroadcastHashJoin [codegen id : 19] Left keys [1]: [c_current_cdemo_sk#43] Right keys [1]: [cd_demo_sk#54] Join type: Inner Join condition: NOT (cd_marital_status#53 = cd_marital_status#55) -(66) Project [codegen id : 24] +(67) Project [codegen id : 19] Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_marital_status#53, cd_demo_sk#54, cd_marital_status#55] -(67) Scan parquet spark_catalog.default.promotion +(68) Scan parquet spark_catalog.default.promotion Output [1]: [p_promo_sk#56] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_promo_sk)] ReadSchema: struct -(68) CometFilter +(69) CometFilter Input [1]: [p_promo_sk#56] Condition : isnotnull(p_promo_sk#56) -(69) ColumnarToRow [codegen id : 16] +(70) ColumnarToRow [codegen id : 11] Input [1]: [p_promo_sk#56] -(70) BroadcastExchange +(71) BroadcastExchange Input [1]: [p_promo_sk#56] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(71) BroadcastHashJoin [codegen id : 24] +(72) BroadcastHashJoin [codegen id : 19] Left keys [1]: [ss_promo_sk#7] Right keys [1]: [p_promo_sk#56] Join type: Inner Join condition: None -(72) Project [codegen id : 24] +(73) Project [codegen id : 19] Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, p_promo_sk#56] -(73) Scan parquet spark_catalog.default.household_demographics +(74) Scan parquet spark_catalog.default.household_demographics Output [2]: [hd_demo_sk#57, hd_income_band_sk#58] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] ReadSchema: struct -(74) CometFilter +(75) CometFilter Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] Condition : (isnotnull(hd_demo_sk#57) AND isnotnull(hd_income_band_sk#58)) -(75) ColumnarToRow [codegen id : 17] +(76) ColumnarToRow [codegen id : 12] Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] -(76) BroadcastExchange +(77) BroadcastExchange Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -(77) BroadcastHashJoin [codegen id : 24] +(78) BroadcastHashJoin [codegen id : 19] Left keys [1]: [ss_hdemo_sk#4] Right keys [1]: [hd_demo_sk#57] Join type: Inner Join condition: None -(78) Project [codegen id : 24] +(79) Project [codegen id : 19] Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58] Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_demo_sk#57, hd_income_band_sk#58] -(79) ReusedExchange [Reuses operator id: 76] +(80) ReusedExchange [Reuses operator id: 77] Output [2]: [hd_demo_sk#59, hd_income_band_sk#60] -(80) BroadcastHashJoin [codegen id : 24] +(81) BroadcastHashJoin [codegen id : 19] Left keys [1]: [c_current_hdemo_sk#44] Right keys [1]: [hd_demo_sk#59] Join type: Inner Join condition: None -(81) Project [codegen id : 24] +(82) Project [codegen id : 19] Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60] Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_demo_sk#59, hd_income_band_sk#60] -(82) Scan parquet spark_catalog.default.customer_address +(83) Scan parquet spark_catalog.default.customer_address Output [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(83) CometFilter +(84) CometFilter Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] Condition : isnotnull(ca_address_sk#61) -(84) ColumnarToRow [codegen id : 19] +(85) ColumnarToRow [codegen id : 14] Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -(85) BroadcastExchange +(86) BroadcastExchange Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=11] -(86) BroadcastHashJoin [codegen id : 24] +(87) BroadcastHashJoin [codegen id : 19] Left keys [1]: [ss_addr_sk#5] Right keys [1]: [ca_address_sk#61] Join type: Inner Join condition: None -(87) Project [codegen id : 24] +(88) Project [codegen id : 19] Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -(88) ReusedExchange [Reuses operator id: 85] +(89) ReusedExchange [Reuses operator id: 86] Output [5]: [ca_address_sk#66, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] -(89) BroadcastHashJoin [codegen id : 24] +(90) BroadcastHashJoin [codegen id : 19] Left keys [1]: [c_current_addr_sk#45] Right keys [1]: [ca_address_sk#66] Join type: Inner Join condition: None -(90) Project [codegen id : 24] +(91) Project [codegen id : 19] Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_address_sk#66, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] -(91) Scan parquet spark_catalog.default.income_band +(92) Scan parquet spark_catalog.default.income_band Output [1]: [ib_income_band_sk#71] Batched: true Location [not included in comparison]/{warehouse_dir}/income_band] PushedFilters: [IsNotNull(ib_income_band_sk)] ReadSchema: struct -(92) CometFilter +(93) CometFilter Input [1]: [ib_income_band_sk#71] Condition : isnotnull(ib_income_band_sk#71) -(93) ColumnarToRow [codegen id : 21] +(94) ColumnarToRow [codegen id : 16] Input [1]: [ib_income_band_sk#71] -(94) BroadcastExchange +(95) BroadcastExchange Input [1]: [ib_income_band_sk#71] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -(95) BroadcastHashJoin [codegen id : 24] +(96) BroadcastHashJoin [codegen id : 19] Left keys [1]: [hd_income_band_sk#58] Right keys [1]: [ib_income_band_sk#71] Join type: Inner Join condition: None -(96) Project [codegen id : 24] +(97) Project [codegen id : 19] Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, ib_income_band_sk#71] -(97) ReusedExchange [Reuses operator id: 94] +(98) ReusedExchange [Reuses operator id: 95] Output [1]: [ib_income_band_sk#72] -(98) BroadcastHashJoin [codegen id : 24] +(99) BroadcastHashJoin [codegen id : 19] Left keys [1]: [hd_income_band_sk#60] Right keys [1]: [ib_income_band_sk#72] Join type: Inner Join condition: None -(99) Project [codegen id : 24] +(100) Project [codegen id : 19] Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, ib_income_band_sk#72] -(100) Scan parquet spark_catalog.default.item +(101) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), In(i_color, [burlywood ,floral ,indian ,medium ,purple ,spring ]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] ReadSchema: struct -(101) CometFilter +(102) CometFilter Input [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] Condition : ((((((isnotnull(i_current_price#74) AND i_color#75 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#74 >= 64.00)) AND (i_current_price#74 <= 74.00)) AND (i_current_price#74 >= 65.00)) AND (i_current_price#74 <= 79.00)) AND isnotnull(i_item_sk#73)) -(102) CometProject +(103) CometProject Input [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] Arguments: [i_item_sk#73, i_product_name#76], [i_item_sk#73, i_product_name#76] -(103) ColumnarToRow [codegen id : 23] +(104) ColumnarToRow [codegen id : 18] Input [2]: [i_item_sk#73, i_product_name#76] -(104) BroadcastExchange +(105) BroadcastExchange Input [2]: [i_item_sk#73, i_product_name#76] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -(105) BroadcastHashJoin [codegen id : 24] +(106) BroadcastHashJoin [codegen id : 19] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#73] Join type: Inner Join condition: None -(106) Project [codegen id : 24] +(107) Project [codegen id : 19] Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, d_year#49, d_year#51, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] -(107) HashAggregate [codegen id : 24] +(108) HashAggregate [codegen id : 19] Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, d_year#49, d_year#51, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] Keys [15]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51] Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] Aggregate Attributes [4]: [count#77, sum#78, sum#79, sum#80] Results [19]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51, count#81, sum#82, sum#83, sum#84] -(108) HashAggregate [codegen id : 24] +(109) HashAggregate [codegen id : 19] Input [19]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51, count#81, sum#82, sum#83, sum#84] Keys [15]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51] Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] Aggregate Attributes [4]: [count(1)#85, sum(UnscaledValue(ss_wholesale_cost#9))#86, sum(UnscaledValue(ss_list_price#10))#87, sum(UnscaledValue(ss_coupon_amt#11))#88] Results [17]: [i_product_name#76 AS product_name#89, i_item_sk#73 AS item_sk#90, s_store_name#40 AS store_name#91, s_zip#41 AS store_zip#92, ca_street_number#62 AS b_street_number#93, ca_street_name#63 AS b_streen_name#94, ca_city#64 AS b_city#95, ca_zip#65 AS b_zip#96, ca_street_number#67 AS c_street_number#97, ca_street_name#68 AS c_street_name#98, ca_city#69 AS c_city#99, ca_zip#70 AS c_zip#100, d_year#38 AS syear#101, count(1)#85 AS cnt#102, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#86,17,2) AS s1#103, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#87,17,2) AS s2#104, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#88,17,2) AS s3#105] -(109) Exchange +(110) RowToColumnar +Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] + +(111) CometColumnarExchange Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] -Arguments: hashpartitioning(item_sk#90, store_name#91, store_zip#92, 5), ENSURE_REQUIREMENTS, [plan_id=14] +Arguments: hashpartitioning(item_sk#90, store_name#91, store_zip#92, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] -(110) Sort [codegen id : 25] +(112) CometSort Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] -Arguments: [item_sk#90 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, store_zip#92 ASC NULLS FIRST], false, 0 +Arguments: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105], [item_sk#90 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, store_zip#92 ASC NULLS FIRST] -(111) Scan parquet spark_catalog.default.store_sales +(113) ColumnarToRow [codegen id : 20] +Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] + +(114) Scan parquet spark_catalog.default.store_sales 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] Batched: true Location: InMemoryFileIndex [] @@ -699,365 +715,380 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#117), dynamicpruningexpression(ss_s PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct -(112) CometFilter +(115) CometFilter Input [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] Condition : (((((((isnotnull(ss_item_sk#106) AND isnotnull(ss_ticket_number#113)) AND isnotnull(ss_store_sk#111)) AND isnotnull(ss_customer_sk#107)) AND isnotnull(ss_cdemo_sk#108)) AND isnotnull(ss_promo_sk#112)) AND isnotnull(ss_hdemo_sk#109)) AND isnotnull(ss_addr_sk#110)) -(113) CometBroadcastExchange +(116) CometBroadcastExchange Input [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] Arguments: [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] -(114) Scan parquet spark_catalog.default.store_returns +(117) Scan parquet spark_catalog.default.store_returns Output [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] ReadSchema: struct -(115) CometFilter +(118) CometFilter Input [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] Condition : (isnotnull(sr_item_sk#119) AND isnotnull(sr_ticket_number#120)) -(116) CometProject +(119) CometProject Input [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] Arguments: [sr_item_sk#119, sr_ticket_number#120], [sr_item_sk#119, sr_ticket_number#120] -(117) CometBroadcastHashJoin +(120) 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, BuildLeft -(118) CometProject +(121) 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] Arguments: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117], [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -(119) ColumnarToRow [codegen id : 26] +(122) CometColumnarExchange Input [11]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Arguments: hashpartitioning(ss_item_sk#106, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] -(120) Exchange +(123) CometSort Input [11]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -Arguments: hashpartitioning(ss_item_sk#106, 5), ENSURE_REQUIREMENTS, [plan_id=15] +Arguments: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117], [ss_item_sk#106 ASC NULLS FIRST] -(121) Sort [codegen id : 27] +(124) ColumnarToRow [codegen id : 21] Input [11]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -Arguments: [ss_item_sk#106 ASC NULLS FIRST], false, 0 -(122) ReusedExchange [Reuses operator id: 27] +(125) ReusedExchange [Reuses operator id: 27] Output [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] -(123) HashAggregate [codegen id : 33] +(126) ColumnarToRow [codegen id : 23] +Input [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] + +(127) HashAggregate [codegen id : 23] Input [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] Keys [1]: [cs_item_sk#122] Functions [2]: [sum(UnscaledValue(cs_ext_list_price#126)), sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#126))#33, sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))#34] Results [3]: [cs_item_sk#122, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#126))#33,17,2) AS sale#130, sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))#34 AS refund#131] -(124) Filter [codegen id : 33] +(128) Filter [codegen id : 23] Input [3]: [cs_item_sk#122, sale#130, refund#131] Condition : ((isnotnull(sale#130) AND isnotnull(refund#131)) AND (cast(sale#130 as decimal(21,2)) > (2 * refund#131))) -(125) Project [codegen id : 33] +(129) Project [codegen id : 23] Output [1]: [cs_item_sk#122] Input [3]: [cs_item_sk#122, sale#130, refund#131] -(126) Sort [codegen id : 33] +(130) Sort [codegen id : 23] Input [1]: [cs_item_sk#122] Arguments: [cs_item_sk#122 ASC NULLS FIRST], false, 0 -(127) SortMergeJoin [codegen id : 49] +(131) SortMergeJoin [codegen id : 39] Left keys [1]: [ss_item_sk#106] Right keys [1]: [cs_item_sk#122] Join type: Inner Join condition: None -(128) Project [codegen id : 49] +(132) Project [codegen id : 39] Output [11]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] Input [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, cs_item_sk#122] -(129) ReusedExchange [Reuses operator id: 189] +(133) ReusedExchange [Reuses operator id: 197] Output [2]: [d_date_sk#132, d_year#133] -(130) BroadcastHashJoin [codegen id : 49] +(134) BroadcastHashJoin [codegen id : 39] Left keys [1]: [ss_sold_date_sk#117] Right keys [1]: [d_date_sk#132] Join type: Inner Join condition: None -(131) Project [codegen id : 49] +(135) Project [codegen id : 39] Output [11]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133] Input [13]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, d_date_sk#132, d_year#133] -(132) ReusedExchange [Reuses operator id: 40] +(136) ReusedExchange [Reuses operator id: 41] Output [3]: [s_store_sk#134, s_store_name#135, s_zip#136] -(133) BroadcastHashJoin [codegen id : 49] +(137) BroadcastHashJoin [codegen id : 39] Left keys [1]: [ss_store_sk#111] Right keys [1]: [s_store_sk#134] Join type: Inner Join condition: None -(134) Project [codegen id : 49] +(138) Project [codegen id : 39] Output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136] 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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_sk#134, s_store_name#135, s_zip#136] -(135) ReusedExchange [Reuses operator id: 46] +(139) ReusedExchange [Reuses operator id: 47] Output [6]: [c_customer_sk#137, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, c_first_shipto_date_sk#141, c_first_sales_date_sk#142] -(136) BroadcastHashJoin [codegen id : 49] +(140) BroadcastHashJoin [codegen id : 39] Left keys [1]: [ss_customer_sk#107] Right keys [1]: [c_customer_sk#137] Join type: Inner Join condition: None -(137) Project [codegen id : 49] +(141) Project [codegen id : 39] Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, c_first_shipto_date_sk#141, c_first_sales_date_sk#142] Input [18]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_customer_sk#137, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, c_first_shipto_date_sk#141, c_first_sales_date_sk#142] -(138) ReusedExchange [Reuses operator id: 52] +(142) ReusedExchange [Reuses operator id: 53] Output [2]: [d_date_sk#143, d_year#144] -(139) BroadcastHashJoin [codegen id : 49] +(143) BroadcastHashJoin [codegen id : 39] Left keys [1]: [c_first_sales_date_sk#142] Right keys [1]: [d_date_sk#143] Join type: Inner Join condition: None -(140) Project [codegen id : 49] +(144) Project [codegen id : 39] Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, c_first_shipto_date_sk#141, d_year#144] Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, c_first_shipto_date_sk#141, c_first_sales_date_sk#142, d_date_sk#143, d_year#144] -(141) ReusedExchange [Reuses operator id: 52] +(145) ReusedExchange [Reuses operator id: 53] Output [2]: [d_date_sk#145, d_year#146] -(142) BroadcastHashJoin [codegen id : 49] +(146) BroadcastHashJoin [codegen id : 39] Left keys [1]: [c_first_shipto_date_sk#141] Right keys [1]: [d_date_sk#145] Join type: Inner Join condition: None -(143) Project [codegen id : 49] +(147) Project [codegen id : 39] Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146] Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, c_first_shipto_date_sk#141, d_year#144, d_date_sk#145, d_year#146] -(144) ReusedExchange [Reuses operator id: 61] +(148) ReusedExchange [Reuses operator id: 62] Output [2]: [cd_demo_sk#147, cd_marital_status#148] -(145) BroadcastHashJoin [codegen id : 49] +(149) BroadcastHashJoin [codegen id : 39] Left keys [1]: [ss_cdemo_sk#108] Right keys [1]: [cd_demo_sk#147] Join type: Inner Join condition: None -(146) Project [codegen id : 49] +(150) Project [codegen id : 39] Output [16]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146, cd_marital_status#148] Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146, cd_demo_sk#147, cd_marital_status#148] -(147) ReusedExchange [Reuses operator id: 61] +(151) ReusedExchange [Reuses operator id: 62] Output [2]: [cd_demo_sk#149, cd_marital_status#150] -(148) BroadcastHashJoin [codegen id : 49] +(152) BroadcastHashJoin [codegen id : 39] Left keys [1]: [c_current_cdemo_sk#138] Right keys [1]: [cd_demo_sk#149] Join type: Inner Join condition: NOT (cd_marital_status#148 = cd_marital_status#150) -(149) Project [codegen id : 49] +(153) Project [codegen id : 39] Output [14]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146] Input [18]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146, cd_marital_status#148, cd_demo_sk#149, cd_marital_status#150] -(150) ReusedExchange [Reuses operator id: 70] +(154) ReusedExchange [Reuses operator id: 71] Output [1]: [p_promo_sk#151] -(151) BroadcastHashJoin [codegen id : 49] +(155) BroadcastHashJoin [codegen id : 39] Left keys [1]: [ss_promo_sk#112] Right keys [1]: [p_promo_sk#151] Join type: Inner Join condition: None -(152) Project [codegen id : 49] +(156) Project [codegen id : 39] Output [13]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146] Input [15]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146, p_promo_sk#151] -(153) ReusedExchange [Reuses operator id: 76] +(157) ReusedExchange [Reuses operator id: 77] Output [2]: [hd_demo_sk#152, hd_income_band_sk#153] -(154) BroadcastHashJoin [codegen id : 49] +(158) BroadcastHashJoin [codegen id : 39] Left keys [1]: [ss_hdemo_sk#109] Right keys [1]: [hd_demo_sk#152] Join type: Inner Join condition: None -(155) Project [codegen id : 49] +(159) Project [codegen id : 39] Output [13]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146, hd_income_band_sk#153] Input [15]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146, hd_demo_sk#152, hd_income_band_sk#153] -(156) ReusedExchange [Reuses operator id: 76] +(160) ReusedExchange [Reuses operator id: 77] Output [2]: [hd_demo_sk#154, hd_income_band_sk#155] -(157) BroadcastHashJoin [codegen id : 49] +(161) BroadcastHashJoin [codegen id : 39] Left keys [1]: [c_current_hdemo_sk#139] Right keys [1]: [hd_demo_sk#154] Join type: Inner Join condition: None -(158) Project [codegen id : 49] +(162) Project [codegen id : 39] Output [13]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_addr_sk#140, d_year#144, d_year#146, hd_income_band_sk#153, hd_income_band_sk#155] Input [15]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146, hd_income_band_sk#153, hd_demo_sk#154, hd_income_band_sk#155] -(159) ReusedExchange [Reuses operator id: 85] +(163) ReusedExchange [Reuses operator id: 86] Output [5]: [ca_address_sk#156, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160] -(160) BroadcastHashJoin [codegen id : 49] +(164) BroadcastHashJoin [codegen id : 39] Left keys [1]: [ss_addr_sk#110] Right keys [1]: [ca_address_sk#156] Join type: Inner Join condition: None -(161) Project [codegen id : 49] +(165) Project [codegen id : 39] Output [16]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_addr_sk#140, d_year#144, d_year#146, hd_income_band_sk#153, hd_income_band_sk#155, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160] Input [18]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_addr_sk#140, d_year#144, d_year#146, hd_income_band_sk#153, hd_income_band_sk#155, ca_address_sk#156, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160] -(162) ReusedExchange [Reuses operator id: 85] +(166) ReusedExchange [Reuses operator id: 86] Output [5]: [ca_address_sk#161, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165] -(163) BroadcastHashJoin [codegen id : 49] +(167) BroadcastHashJoin [codegen id : 39] Left keys [1]: [c_current_addr_sk#140] Right keys [1]: [ca_address_sk#161] Join type: Inner Join condition: None -(164) Project [codegen id : 49] +(168) Project [codegen id : 39] Output [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, d_year#144, d_year#146, hd_income_band_sk#153, hd_income_band_sk#155, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165] Input [21]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_addr_sk#140, d_year#144, d_year#146, hd_income_band_sk#153, hd_income_band_sk#155, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_address_sk#161, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165] -(165) ReusedExchange [Reuses operator id: 94] +(169) ReusedExchange [Reuses operator id: 95] Output [1]: [ib_income_band_sk#166] -(166) BroadcastHashJoin [codegen id : 49] +(170) BroadcastHashJoin [codegen id : 39] Left keys [1]: [hd_income_band_sk#153] Right keys [1]: [ib_income_band_sk#166] Join type: Inner Join condition: None -(167) Project [codegen id : 49] +(171) Project [codegen id : 39] Output [18]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, d_year#144, d_year#146, hd_income_band_sk#155, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165] Input [20]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, d_year#144, d_year#146, hd_income_band_sk#153, hd_income_band_sk#155, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, ib_income_band_sk#166] -(168) ReusedExchange [Reuses operator id: 94] +(172) ReusedExchange [Reuses operator id: 95] Output [1]: [ib_income_band_sk#167] -(169) BroadcastHashJoin [codegen id : 49] +(173) BroadcastHashJoin [codegen id : 39] Left keys [1]: [hd_income_band_sk#155] Right keys [1]: [ib_income_band_sk#167] Join type: Inner Join condition: None -(170) Project [codegen id : 49] +(174) Project [codegen id : 39] Output [17]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, d_year#144, d_year#146, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165] Input [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, d_year#144, d_year#146, hd_income_band_sk#155, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, ib_income_band_sk#167] -(171) ReusedExchange [Reuses operator id: 104] +(175) ReusedExchange [Reuses operator id: 105] Output [2]: [i_item_sk#168, i_product_name#169] -(172) BroadcastHashJoin [codegen id : 49] +(176) BroadcastHashJoin [codegen id : 39] Left keys [1]: [ss_item_sk#106] Right keys [1]: [i_item_sk#168] Join type: Inner Join condition: None -(173) Project [codegen id : 49] +(177) Project [codegen id : 39] Output [18]: [ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, d_year#144, d_year#146, s_store_name#135, s_zip#136, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, i_item_sk#168, i_product_name#169] Input [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, d_year#144, d_year#146, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, i_item_sk#168, i_product_name#169] -(174) HashAggregate [codegen id : 49] +(178) HashAggregate [codegen id : 39] Input [18]: [ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, d_year#144, d_year#146, s_store_name#135, s_zip#136, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, i_item_sk#168, i_product_name#169] Keys [15]: [i_product_name#169, i_item_sk#168, s_store_name#135, s_zip#136, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, d_year#133, d_year#144, d_year#146] Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#114)), partial_sum(UnscaledValue(ss_list_price#115)), partial_sum(UnscaledValue(ss_coupon_amt#116))] Aggregate Attributes [4]: [count#77, sum#170, sum#171, sum#172] Results [19]: [i_product_name#169, i_item_sk#168, s_store_name#135, s_zip#136, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, d_year#133, d_year#144, d_year#146, count#81, sum#173, sum#174, sum#175] -(175) HashAggregate [codegen id : 49] +(179) HashAggregate [codegen id : 39] Input [19]: [i_product_name#169, i_item_sk#168, s_store_name#135, s_zip#136, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, d_year#133, d_year#144, d_year#146, count#81, sum#173, sum#174, sum#175] Keys [15]: [i_product_name#169, i_item_sk#168, s_store_name#135, s_zip#136, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, d_year#133, d_year#144, d_year#146] Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#114)), sum(UnscaledValue(ss_list_price#115)), sum(UnscaledValue(ss_coupon_amt#116))] Aggregate Attributes [4]: [count(1)#85, sum(UnscaledValue(ss_wholesale_cost#114))#86, sum(UnscaledValue(ss_list_price#115))#87, sum(UnscaledValue(ss_coupon_amt#116))#88] Results [8]: [i_item_sk#168 AS item_sk#176, s_store_name#135 AS store_name#177, s_zip#136 AS store_zip#178, d_year#133 AS syear#179, count(1)#85 AS cnt#180, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#114))#86,17,2) AS s1#181, MakeDecimal(sum(UnscaledValue(ss_list_price#115))#87,17,2) AS s2#182, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#116))#88,17,2) AS s3#183] -(176) Exchange +(180) RowToColumnar +Input [8]: [item_sk#176, store_name#177, store_zip#178, syear#179, cnt#180, s1#181, s2#182, s3#183] + +(181) CometColumnarExchange +Input [8]: [item_sk#176, store_name#177, store_zip#178, syear#179, cnt#180, s1#181, s2#182, s3#183] +Arguments: hashpartitioning(item_sk#176, store_name#177, store_zip#178, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] + +(182) CometSort Input [8]: [item_sk#176, store_name#177, store_zip#178, syear#179, cnt#180, s1#181, s2#182, s3#183] -Arguments: hashpartitioning(item_sk#176, store_name#177, store_zip#178, 5), ENSURE_REQUIREMENTS, [plan_id=16] +Arguments: [item_sk#176, store_name#177, store_zip#178, syear#179, cnt#180, s1#181, s2#182, s3#183], [item_sk#176 ASC NULLS FIRST, store_name#177 ASC NULLS FIRST, store_zip#178 ASC NULLS FIRST] -(177) Sort [codegen id : 50] +(183) ColumnarToRow [codegen id : 40] Input [8]: [item_sk#176, store_name#177, store_zip#178, syear#179, cnt#180, s1#181, s2#182, s3#183] -Arguments: [item_sk#176 ASC NULLS FIRST, store_name#177 ASC NULLS FIRST, store_zip#178 ASC NULLS FIRST], false, 0 -(178) SortMergeJoin [codegen id : 51] +(184) SortMergeJoin [codegen id : 41] Left keys [3]: [item_sk#90, store_name#91, store_zip#92] Right keys [3]: [item_sk#176, store_name#177, store_zip#178] Join type: Inner Join condition: (cnt#180 <= cnt#102) -(179) Project [codegen id : 51] +(185) Project [codegen id : 41] Output [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#181, s2#182, s3#183, syear#179, cnt#180] Input [25]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, item_sk#176, store_name#177, store_zip#178, syear#179, cnt#180, s1#181, s2#182, s3#183] -(180) Exchange +(186) RowToColumnar +Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#181, s2#182, s3#183, syear#179, cnt#180] + +(187) CometColumnarExchange +Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#181, s2#182, s3#183, syear#179, cnt#180] +Arguments: rangepartitioning(product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#180 ASC NULLS FIRST, s1#103 ASC NULLS FIRST, s1#181 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] + +(188) CometSort Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#181, s2#182, s3#183, syear#179, cnt#180] -Arguments: rangepartitioning(product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#180 ASC NULLS FIRST, s1#103 ASC NULLS FIRST, s1#181 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=17] +Arguments: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#181, s2#182, s3#183, syear#179, cnt#180], [product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#180 ASC NULLS FIRST, s1#103 ASC NULLS FIRST, s1#181 ASC NULLS FIRST] -(181) Sort [codegen id : 52] +(189) ColumnarToRow [codegen id : 42] Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#181, s2#182, s3#183, syear#179, cnt#180] -Arguments: [product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#180 ASC NULLS FIRST, s1#103 ASC NULLS FIRST, s1#181 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (185) -+- * ColumnarToRow (184) - +- CometFilter (183) - +- CometScan parquet spark_catalog.default.date_dim (182) +BroadcastExchange (193) ++- * ColumnarToRow (192) + +- CometFilter (191) + +- CometScan parquet spark_catalog.default.date_dim (190) -(182) Scan parquet spark_catalog.default.date_dim +(190) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#37, d_year#38] 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 -(183) CometFilter +(191) CometFilter Input [2]: [d_date_sk#37, d_year#38] Condition : ((isnotnull(d_year#38) AND (d_year#38 = 1999)) AND isnotnull(d_date_sk#37)) -(184) ColumnarToRow [codegen id : 1] +(192) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#37, d_year#38] -(185) BroadcastExchange +(193) BroadcastExchange Input [2]: [d_date_sk#37, d_year#38] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] -Subquery:2 Hosting operator id = 111 Hosting Expression = ss_sold_date_sk#117 IN dynamicpruning#118 -BroadcastExchange (189) -+- * ColumnarToRow (188) - +- CometFilter (187) - +- CometScan parquet spark_catalog.default.date_dim (186) +Subquery:2 Hosting operator id = 114 Hosting Expression = ss_sold_date_sk#117 IN dynamicpruning#118 +BroadcastExchange (197) ++- * ColumnarToRow (196) + +- CometFilter (195) + +- CometScan parquet spark_catalog.default.date_dim (194) -(186) Scan parquet spark_catalog.default.date_dim +(194) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#132, d_year#133] 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 -(187) CometFilter +(195) CometFilter Input [2]: [d_date_sk#132, d_year#133] Condition : ((isnotnull(d_year#133) AND (d_year#133 = 2000)) AND isnotnull(d_date_sk#132)) -(188) ColumnarToRow [codegen id : 1] +(196) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#132, d_year#133] -(189) BroadcastExchange +(197) BroadcastExchange Input [2]: [d_date_sk#132, d_year#133] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=19] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt index 512a74f9f..09bc3d099 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt @@ -1,281 +1,276 @@ -WholeStageCodegen (52) - Sort [product_name,store_name,cnt,s1,s1] +WholeStageCodegen (42) + ColumnarToRow InputAdapter - Exchange [product_name,store_name,cnt,s1,s1] #1 - WholeStageCodegen (51) - Project [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - SortMergeJoin [item_sk,store_name,store_zip,item_sk,store_name,store_zip,cnt,cnt] - InputAdapter - WholeStageCodegen (25) - Sort [item_sk,store_name,store_zip] - InputAdapter - Exchange [item_sk,store_name,store_zip] #2 - WholeStageCodegen (24) - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] - Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SortMergeJoin [ss_item_sk,cs_item_sk] - InputAdapter - WholeStageCodegen (2) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #3 + CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] + CometColumnarExchange [product_name,store_name,cnt,s1,s1] #1 + RowToColumnar + WholeStageCodegen (41) + Project [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] + SortMergeJoin [item_sk,store_name,store_zip,item_sk,store_name,store_zip,cnt,cnt] + InputAdapter + WholeStageCodegen (20) + ColumnarToRow + InputAdapter + CometSort [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3] + CometColumnarExchange [item_sk,store_name,store_zip] #2 + RowToColumnar + WholeStageCodegen (19) + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] + Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SortMergeJoin [ss_item_sk,cs_item_sk] + InputAdapter WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #4 - CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) + CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk] #3 + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #4 + CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + WholeStageCodegen (3) + Sort [cs_item_sk] + Project [cs_item_sk] + Filter [sale,refund] + HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [cs_item_sk] #6 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [sum,sum,isEmpty,sum,sum,isEmpty] ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometSortMergeJoin [cs_item_sk,cs_order_number,cs_ext_list_price,cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometSort [cs_item_sk,cs_order_number,cs_ext_list_price] + CometColumnarExchange [cs_item_sk,cs_order_number] #7 + CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometColumnarExchange [cr_item_sk,cr_order_number] #8 + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 InputAdapter - WholeStageCodegen (8) - Sort [cs_item_sk] - Project [cs_item_sk] - Filter [sale,refund] - HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] - InputAdapter - Exchange [cs_item_sk] #6 - WholeStageCodegen (7) - HashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [sum,sum,isEmpty,sum,sum,isEmpty] - Project [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] - InputAdapter - WholeStageCodegen (4) - Sort [cs_item_sk,cs_order_number] - InputAdapter - Exchange [cs_item_sk,cs_order_number] #7 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - InputAdapter - WholeStageCodegen (6) - Sort [cr_item_sk,cr_order_number] - InputAdapter - Exchange [cr_item_sk,cr_order_number] #8 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + BroadcastExchange #9 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk,s_store_name,s_zip] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] InputAdapter - ReusedExchange [d_date_sk,d_year] #5 + BroadcastExchange #10 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (10) + BroadcastExchange #11 + WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [s_store_sk,s_store_name,s_zip] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - BroadcastExchange #10 - WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + ReusedExchange [d_date_sk,d_year] #11 InputAdapter - BroadcastExchange #11 - WholeStageCodegen (12) + BroadcastExchange #12 + WholeStageCodegen (9) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometFilter [cd_demo_sk,cd_marital_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] InputAdapter - ReusedExchange [d_date_sk,d_year] #11 + ReusedExchange [cd_demo_sk,cd_marital_status] #12 InputAdapter - BroadcastExchange #12 - WholeStageCodegen (14) + BroadcastExchange #13 + WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk,cd_marital_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + CometFilter [p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk] InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 + BroadcastExchange #14 + WholeStageCodegen (12) + 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 #13 - WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometFilter [p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk] + ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 InputAdapter - BroadcastExchange #14 - WholeStageCodegen (17) + BroadcastExchange #15 + WholeStageCodegen (14) ColumnarToRow InputAdapter - CometFilter [hd_demo_sk,hd_income_band_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 InputAdapter - BroadcastExchange #15 - WholeStageCodegen (19) + BroadcastExchange #16 + WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometFilter [ib_income_band_sk] + CometScan parquet spark_catalog.default.income_band [ib_income_band_sk] InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 + ReusedExchange [ib_income_band_sk] #16 InputAdapter - BroadcastExchange #16 - WholeStageCodegen (21) + BroadcastExchange #17 + WholeStageCodegen (18) ColumnarToRow InputAdapter - CometFilter [ib_income_band_sk] - CometScan parquet spark_catalog.default.income_band [ib_income_band_sk] - InputAdapter - ReusedExchange [ib_income_band_sk] #16 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (23) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] - InputAdapter - WholeStageCodegen (50) - Sort [item_sk,store_name,store_zip] - InputAdapter - Exchange [item_sk,store_name,store_zip] #18 - WholeStageCodegen (49) - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] - Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SortMergeJoin [ss_item_sk,cs_item_sk] - InputAdapter - WholeStageCodegen (27) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #19 - WholeStageCodegen (26) + CometProject [i_item_sk,i_product_name] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] + InputAdapter + WholeStageCodegen (40) + ColumnarToRow + InputAdapter + CometSort [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] + CometColumnarExchange [item_sk,store_name,store_zip] #18 + RowToColumnar + WholeStageCodegen (39) + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] + Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SortMergeJoin [ss_item_sk,cs_item_sk] + InputAdapter + WholeStageCodegen (21) ColumnarToRow InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #20 - CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #21 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk] #19 + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #20 + CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #21 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + WholeStageCodegen (23) + Sort [cs_item_sk] + Project [cs_item_sk] + Filter [sale,refund] + HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] + ColumnarToRow + InputAdapter + ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #21 InputAdapter - WholeStageCodegen (33) - Sort [cs_item_sk] - Project [cs_item_sk] - Filter [sale,refund] - HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] - InputAdapter - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 + ReusedExchange [s_store_sk,s_store_name,s_zip] #9 InputAdapter - ReusedExchange [d_date_sk,d_year] #21 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #10 InputAdapter - ReusedExchange [s_store_sk,s_store_name,s_zip] #9 + ReusedExchange [d_date_sk,d_year] #11 InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #10 + ReusedExchange [d_date_sk,d_year] #11 InputAdapter - ReusedExchange [d_date_sk,d_year] #11 + ReusedExchange [cd_demo_sk,cd_marital_status] #12 InputAdapter - ReusedExchange [d_date_sk,d_year] #11 + ReusedExchange [cd_demo_sk,cd_marital_status] #12 InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 + ReusedExchange [p_promo_sk] #13 InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 + ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 InputAdapter - ReusedExchange [p_promo_sk] #13 + ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 + ReusedExchange [ib_income_band_sk] #16 InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 + ReusedExchange [ib_income_band_sk] #16 InputAdapter - ReusedExchange [ib_income_band_sk] #16 - InputAdapter - ReusedExchange [ib_income_band_sk] #16 - InputAdapter - ReusedExchange [i_item_sk,i_product_name] #17 + ReusedExchange [i_item_sk,i_product_name] #17 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 25e832248..0c59ffbbc 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,75 +1,103 @@ == Physical Plan == -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) +TakeOrderedAndProject (99) ++- * Filter (98) + +- Window (97) + +- WindowGroupLimit (96) + +- * ColumnarToRow (95) + +- CometSort (94) + +- CometColumnarExchange (93) + +- RowToColumnar (92) + +- WindowGroupLimit (91) + +- * Sort (90) + +- Union (89) + :- * HashAggregate (24) + : +- * ColumnarToRow (23) + : +- CometColumnarExchange (22) + : +- RowToColumnar (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 (32) + : +- * ColumnarToRow (31) + : +- CometColumnarExchange (30) + : +- RowToColumnar (29) + : +- * HashAggregate (28) + : +- * HashAggregate (27) + : +- * ColumnarToRow (26) + : +- ReusedExchange (25) + :- * HashAggregate (40) + : +- * ColumnarToRow (39) + : +- CometColumnarExchange (38) + : +- RowToColumnar (37) + : +- * HashAggregate (36) + : +- * HashAggregate (35) + : +- * ColumnarToRow (34) + : +- ReusedExchange (33) + :- * HashAggregate (48) + : +- * ColumnarToRow (47) + : +- CometColumnarExchange (46) + : +- RowToColumnar (45) + : +- * HashAggregate (44) + : +- * HashAggregate (43) + : +- * ColumnarToRow (42) + : +- ReusedExchange (41) + :- * HashAggregate (56) + : +- * ColumnarToRow (55) + : +- CometColumnarExchange (54) + : +- RowToColumnar (53) + : +- * HashAggregate (52) + : +- * HashAggregate (51) + : +- * ColumnarToRow (50) + : +- ReusedExchange (49) + :- * HashAggregate (64) + : +- * ColumnarToRow (63) + : +- CometColumnarExchange (62) + : +- RowToColumnar (61) + : +- * HashAggregate (60) + : +- * HashAggregate (59) + : +- * ColumnarToRow (58) + : +- ReusedExchange (57) + :- * HashAggregate (72) + : +- * ColumnarToRow (71) + : +- CometColumnarExchange (70) + : +- RowToColumnar (69) + : +- * HashAggregate (68) + : +- * HashAggregate (67) + : +- * ColumnarToRow (66) + : +- ReusedExchange (65) + :- * HashAggregate (80) + : +- * ColumnarToRow (79) + : +- CometColumnarExchange (78) + : +- RowToColumnar (77) + : +- * HashAggregate (76) + : +- * HashAggregate (75) + : +- * ColumnarToRow (74) + : +- ReusedExchange (73) + +- * HashAggregate (88) + +- * ColumnarToRow (87) + +- CometColumnarExchange (86) + +- RowToColumnar (85) + +- * HashAggregate (84) + +- * HashAggregate (83) + +- * ColumnarToRow (82) + +- ReusedExchange (81) (1) Scan parquet spark_catalog.default.store_sales @@ -170,304 +198,388 @@ Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as d 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] -(21) Exchange +(21) RowToColumnar 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] -(22) HashAggregate [codegen id : 2] +(22) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(23) ColumnarToRow [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] + +(24) 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))#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] +(25) ReusedExchange [Reuses operator id: 22] 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] +(26) ColumnarToRow [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] + +(27) 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] +(28) 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 +(29) RowToColumnar +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] + +(30) CometColumnarExchange 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] +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, CometColumnarShuffle, [plan_id=2] -(27) HashAggregate [codegen id : 5] +(31) ColumnarToRow [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] + +(32) 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] +(33) ReusedExchange [Reuses operator id: 22] 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] +(34) ColumnarToRow [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] + +(35) 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] +(36) 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 +(37) RowToColumnar 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] +(38) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=3] + +(39) ColumnarToRow [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] + +(40) 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] +(41) ReusedExchange [Reuses operator id: 22] 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] +(42) ColumnarToRow [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] + +(43) 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] +(44) 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 +(45) RowToColumnar +Input [7]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, sum#89, isEmpty#90] + +(46) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=4] + +(47) ColumnarToRow [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] -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] +(48) 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] +(49) ReusedExchange [Reuses operator id: 22] 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] +(50) ColumnarToRow [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] + +(51) 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] +(52) 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 +(53) RowToColumnar +Input [6]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, sum#111, isEmpty#112] + +(54) CometColumnarExchange 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] +Arguments: hashpartitioning(i_category#96, i_class#97, i_brand#98, i_product_name#99, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(42) HashAggregate [codegen id : 14] +(55) ColumnarToRow [codegen id : 14] +Input [6]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, sum#111, isEmpty#112] + +(56) 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] +(57) ReusedExchange [Reuses operator id: 22] 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] +(58) ColumnarToRow [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] + +(59) 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] +(60) 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 +(61) RowToColumnar +Input [5]: [i_category#119, i_class#120, i_brand#121, sum#134, isEmpty#135] + +(62) CometColumnarExchange 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] +Arguments: hashpartitioning(i_category#119, i_class#120, i_brand#121, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(47) HashAggregate [codegen id : 17] +(63) ColumnarToRow [codegen id : 17] +Input [5]: [i_category#119, i_class#120, i_brand#121, sum#134, isEmpty#135] + +(64) 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] +(65) ReusedExchange [Reuses operator id: 22] 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] +(66) ColumnarToRow [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] + +(67) 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] +(68) 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 +(69) RowToColumnar 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] +(70) CometColumnarExchange +Input [4]: [i_category#143, i_class#144, sum#158, isEmpty#159] +Arguments: hashpartitioning(i_category#143, i_class#144, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(71) ColumnarToRow [codegen id : 20] +Input [4]: [i_category#143, i_class#144, sum#158, isEmpty#159] + +(72) 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] +(73) ReusedExchange [Reuses operator id: 22] 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] +(74) ColumnarToRow [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] + +(75) 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] +(76) 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 +(77) RowToColumnar +Input [3]: [i_category#168, sum#183, isEmpty#184] + +(78) CometColumnarExchange +Input [3]: [i_category#168, sum#183, isEmpty#184] +Arguments: hashpartitioning(i_category#168, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(79) ColumnarToRow [codegen id : 23] 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] +(80) 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] +(81) ReusedExchange [Reuses operator id: 22] 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] +(82) ColumnarToRow [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] + +(83) 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] +(84) HashAggregate [codegen id : 25] Input [1]: [sumsales#206] Keys: [] Functions [1]: [partial_sum(sumsales#206)] Aggregate Attributes [2]: [sum#207, isEmpty#208] Results [2]: [sum#209, isEmpty#210] -(61) Exchange +(85) RowToColumnar +Input [2]: [sum#209, isEmpty#210] + +(86) CometColumnarExchange Input [2]: [sum#209, isEmpty#210] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(62) HashAggregate [codegen id : 26] +(87) ColumnarToRow [codegen id : 26] +Input [2]: [sum#209, isEmpty#210] + +(88) HashAggregate [codegen id : 26] Input [2]: [sum#209, isEmpty#210] Keys: [] 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] -(63) Union +(89) Union -(64) Sort [codegen id : 27] +(90) 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 -(65) WindowGroupLimit +(91) 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 -(66) Exchange +(92) RowToColumnar 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] -(67) Sort [codegen id : 28] +(93) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=10] + +(94) CometSort +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, 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], [i_category#24 ASC NULLS FIRST, sumsales#32 DESC NULLS LAST] + +(95) ColumnarToRow [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 -(68) WindowGroupLimit +(96) 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 -(69) Window +(97) 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] -(70) Filter [codegen id : 29] +(98) 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) -(71) TakeOrderedAndProject +(99) 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 (76) -+- * ColumnarToRow (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan parquet spark_catalog.default.date_dim (72) +BroadcastExchange (104) ++- * ColumnarToRow (103) + +- CometProject (102) + +- CometFilter (101) + +- CometScan parquet spark_catalog.default.date_dim (100) -(72) Scan parquet spark_catalog.default.date_dim +(100) 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 -(73) CometFilter +(101) 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)) -(74) CometProject +(102) 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] -(75) ColumnarToRow [codegen id : 1] +(103) ColumnarToRow [codegen id : 1] Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(76) BroadcastExchange +(104) 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 cc9627e89..00b800f52 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 @@ -5,117 +5,145 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ Window [sumsales,i_category] WindowGroupLimit [i_category,sumsales] WholeStageCodegen (28) - Sort [i_category,sumsales] + ColumnarToRow InputAdapter - Exchange [i_category] #1 - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (27) - Sort [i_category,sumsales] - InputAdapter - Union - 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 (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] - 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,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] - CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] - CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometFilter [s_store_sk,s_store_id] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - 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] #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 (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] #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 (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] #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 (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] #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 (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] #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 (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] #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 (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] #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 (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 #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 - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometColumnarExchange [i_category] #1 + RowToColumnar + WindowGroupLimit [i_category,sumsales] + WholeStageCodegen (27) + Sort [i_category,sumsales] + InputAdapter + Union + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 + RowToColumnar + 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] + 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,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] + CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #7 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #8 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year] #9 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class,i_brand,i_product_name] #10 + RowToColumnar + 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] + ColumnarToRow + 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) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class,i_brand] #11 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class] #12 + RowToColumnar + 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] + ColumnarToRow + 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) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category] #13 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange #14 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 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 90e9a457b..8d8b23eae 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,62 +1,74 @@ == Physical Plan == -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) +TakeOrderedAndProject (70) ++- * Project (69) + +- Window (68) + +- * ColumnarToRow (67) + +- CometSort (66) + +- CometColumnarExchange (65) + +- CometHashAggregate (64) + +- CometColumnarExchange (63) + +- RowToColumnar (62) + +- * HashAggregate (61) + +- Union (60) + :- * HashAggregate (43) + : +- * ColumnarToRow (42) + : +- CometColumnarExchange (41) + : +- RowToColumnar (40) + : +- * HashAggregate (39) + : +- * Project (38) + : +- * BroadcastHashJoin Inner BuildRight (37) + : :- * 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 (36) + : +- * BroadcastHashJoin LeftSemi BuildRight (35) + : :- * ColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.store (10) + : +- BroadcastExchange (34) + : +- * Project (33) + : +- * Filter (32) + : +- Window (31) + : +- WindowGroupLimit (30) + : +- * Sort (29) + : +- * HashAggregate (28) + : +- * ColumnarToRow (27) + : +- CometColumnarExchange (26) + : +- RowToColumnar (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 (51) + : +- * ColumnarToRow (50) + : +- CometColumnarExchange (49) + : +- RowToColumnar (48) + : +- * HashAggregate (47) + : +- * HashAggregate (46) + : +- * ColumnarToRow (45) + : +- ReusedExchange (44) + +- * HashAggregate (59) + +- * ColumnarToRow (58) + +- CometColumnarExchange (57) + +- RowToColumnar (56) + +- * HashAggregate (55) + +- * HashAggregate (54) + +- * ColumnarToRow (53) + +- ReusedExchange (52) (1) Scan parquet spark_catalog.default.store_sales @@ -174,204 +186,238 @@ Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] Aggregate Attributes [1]: [sum#17] Results [2]: [s_state#15, sum#18] -(25) Exchange +(25) RowToColumnar Input [2]: [s_state#15, sum#18] -Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(26) HashAggregate [codegen id : 2] +(26) CometColumnarExchange +Input [2]: [s_state#15, sum#18] +Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(27) ColumnarToRow [codegen id : 2] +Input [2]: [s_state#15, sum#18] + +(28) 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] +(29) 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 -(28) WindowGroupLimit +(30) 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 +(31) 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] +(32) 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] +(33) Project [codegen id : 3] Output [1]: [s_state#15] Input [4]: [s_state#15, _w0#20, s_state#15, ranking#21] -(32) BroadcastExchange +(34) BroadcastExchange Input [1]: [s_state#15] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] -(33) BroadcastHashJoin [codegen id : 4] +(35) BroadcastHashJoin [codegen id : 4] Left keys [1]: [s_state#9] Right keys [1]: [s_state#15] Join type: LeftSemi Join condition: None -(34) BroadcastExchange +(36) 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] -(35) BroadcastHashJoin [codegen id : 5] +(37) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_store_sk#1] Right keys [1]: [s_store_sk#7] Join type: Inner Join condition: None -(36) Project [codegen id : 5] +(38) 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] -(37) HashAggregate [codegen id : 5] +(39) 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#22] Results [3]: [s_state#9, s_county#8, sum#23] -(38) Exchange +(40) RowToColumnar +Input [3]: [s_state#9, s_county#8, sum#23] + +(41) CometColumnarExchange +Input [3]: [s_state#9, s_county#8, sum#23] +Arguments: hashpartitioning(s_state#9, s_county#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(42) ColumnarToRow [codegen id : 6] Input [3]: [s_state#9, s_county#8, sum#23] -Arguments: hashpartitioning(s_state#9, s_county#8, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(39) HashAggregate [codegen id : 6] +(43) 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))#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] +(44) ReusedExchange [Reuses operator id: 41] Output [3]: [s_state#29, s_county#30, sum#31] -(41) HashAggregate [codegen id : 12] +(45) ColumnarToRow [codegen id : 12] +Input [3]: [s_state#29, s_county#30, sum#31] + +(46) 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] +(47) 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 +(48) RowToColumnar +Input [3]: [s_state#29, sum#36, isEmpty#37] + +(49) CometColumnarExchange Input [3]: [s_state#29, sum#36, isEmpty#37] -Arguments: hashpartitioning(s_state#29, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Arguments: hashpartitioning(s_state#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(44) HashAggregate [codegen id : 13] +(50) ColumnarToRow [codegen id : 13] +Input [3]: [s_state#29, sum#36, isEmpty#37] + +(51) 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] +(52) ReusedExchange [Reuses operator id: 41] Output [3]: [s_state#44, s_county#45, sum#46] -(46) HashAggregate [codegen id : 19] +(53) ColumnarToRow [codegen id : 19] +Input [3]: [s_state#44, s_county#45, sum#46] + +(54) 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] +(55) HashAggregate [codegen id : 19] Input [1]: [total_sum#48] Keys: [] Functions [1]: [partial_sum(total_sum#48)] Aggregate Attributes [2]: [sum#49, isEmpty#50] Results [2]: [sum#51, isEmpty#52] -(48) Exchange +(56) RowToColumnar Input [2]: [sum#51, isEmpty#52] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(49) HashAggregate [codegen id : 20] +(57) CometColumnarExchange +Input [2]: [sum#51, isEmpty#52] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(58) ColumnarToRow [codegen id : 20] +Input [2]: [sum#51, isEmpty#52] + +(59) HashAggregate [codegen id : 20] Input [2]: [sum#51, isEmpty#52] Keys: [] 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] -(50) Union +(60) Union -(51) HashAggregate [codegen id : 21] +(61) 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#25, s_state#9, s_county#8, g_state#26, g_county#27, lochierarchy#28] -(52) Exchange +(62) RowToColumnar +Input [6]: [total_sum#25, s_state#9, s_county#8, g_state#26, g_county#27, lochierarchy#28] + +(63) CometColumnarExchange 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] +Arguments: hashpartitioning(total_sum#25, s_state#9, s_county#8, g_state#26, g_county#27, lochierarchy#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(53) HashAggregate [codegen id : 22] +(64) CometHashAggregate 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#25, s_state#9, s_county#8, lochierarchy#28, CASE WHEN (g_county#27 = 0) THEN s_state#9 END AS _w0#60] -(54) Exchange +(65) CometColumnarExchange +Input [5]: [total_sum#25, s_state#9, s_county#8, lochierarchy#28, _w0#60] +Arguments: hashpartitioning(lochierarchy#28, _w0#60, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(66) CometSort 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] +Arguments: [total_sum#25, s_state#9, s_county#8, lochierarchy#28, _w0#60], [lochierarchy#28 ASC NULLS FIRST, _w0#60 ASC NULLS FIRST, total_sum#25 DESC NULLS LAST] -(55) Sort [codegen id : 23] +(67) ColumnarToRow [codegen id : 22] 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 -(56) Window +(68) 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] -(57) Project [codegen id : 24] +(69) Project [codegen id : 23] 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] -(58) TakeOrderedAndProject +(70) 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 (63) -+- * ColumnarToRow (62) - +- CometProject (61) - +- CometFilter (60) - +- CometScan parquet spark_catalog.default.date_dim (59) +BroadcastExchange (75) ++- * ColumnarToRow (74) + +- CometProject (73) + +- CometFilter (72) + +- CometScan parquet spark_catalog.default.date_dim (71) -(59) Scan parquet spark_catalog.default.date_dim +(71) 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 -(60) CometFilter +(72) 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)) -(61) CometProject +(73) CometProject Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(62) ColumnarToRow [codegen id : 1] +(74) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(63) BroadcastExchange +(75) BroadcastExchange Input [1]: [d_date_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] 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 67f4841b4..816cb34fb 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,97 +1,107 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (24) + WholeStageCodegen (23) Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] InputAdapter Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (23) - Sort [lochierarchy,_w0,total_sum] + WholeStageCodegen (22) + ColumnarToRow InputAdapter - Exchange [lochierarchy,_w0] #1 - 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 + CometSort [total_sum,s_state,s_county,lochierarchy,_w0] + CometColumnarExchange [lochierarchy,_w0] #1 + CometHashAggregate [total_sum,s_state,s_county,lochierarchy,_w0,g_state,g_county] + CometColumnarExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 + RowToColumnar WholeStageCodegen (21) HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] InputAdapter Union 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 (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] - ColumnarToRow - InputAdapter - CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - BroadcastHashJoin [s_state,s_state] - ColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (2) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - InputAdapter - Exchange [s_state] #8 - WholeStageCodegen (1) - HashAggregate [s_state,ss_net_profit] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] - CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #9 - CometFilter [s_store_sk,s_state] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [d_date_sk] #5 + ColumnarToRow + InputAdapter + CometColumnarExchange [s_state,s_county] #3 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + CometProject [ss_store_sk,ss_net_profit] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (4) + BroadcastHashJoin [s_state,s_state] + ColumnarToRow + InputAdapter + CometFilter [s_store_sk,s_county,s_state] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (3) + Project [s_state] + Filter [ranking] + InputAdapter + Window [_w0,s_state] + WindowGroupLimit [s_state,_w0] + WholeStageCodegen (2) + Sort [s_state,_w0] + HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [s_state] #8 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [s_state,ss_net_profit] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [ss_net_profit,s_state] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] + CometProject [ss_net_profit,ss_sold_date_sk,s_state] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [s_store_sk,s_state] #9 + CometFilter [s_store_sk,s_state] + 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] #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 + ColumnarToRow + InputAdapter + CometColumnarExchange [s_state] #10 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + ReusedExchange [s_state,s_county,sum] #3 WholeStageCodegen (20) HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - InputAdapter - 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 - ReusedExchange [s_state,s_county,sum] #3 + ColumnarToRow + InputAdapter + CometColumnarExchange #11 + RowToColumnar + WholeStageCodegen (19) + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] + ColumnarToRow + InputAdapter + ReusedExchange [s_state,s_county,sum] #3 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 8bff19a72..53e1bb5a6 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,71 +1,72 @@ == Physical Plan == -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) +* ColumnarToRow (68) ++- CometTakeOrderedAndProject (67) + +- CometHashAggregate (66) + +- CometColumnarExchange (65) + +- CometHashAggregate (64) + +- CometProject (63) + +- CometSortMergeJoin (62) + :- CometSort (56) + : +- CometColumnarExchange (55) + : +- RowToColumnar (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) + +- CometSort (61) + +- CometColumnarExchange (60) + +- CometProject (59) + +- CometFilter (58) + +- CometScan parquet spark_catalog.default.catalog_returns (57) (1) Scan parquet spark_catalog.default.catalog_sales @@ -320,101 +321,99 @@ Join condition: None 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#30] -(54) Exchange +(54) RowToColumnar 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=3] -(55) Sort [codegen id : 4] +(55) CometColumnarExchange 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 +Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(56) Scan parquet spark_catalog.default.catalog_returns +(56) CometSort +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, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST] + +(57) 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 -(57) CometFilter +(58) 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)) -(58) CometProject +(59) 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] -(59) ColumnarToRow [codegen id : 5] +(60) CometColumnarExchange Input [2]: [cr_item_sk#31, cr_order_number#32] +Arguments: hashpartitioning(cr_item_sk#31, cr_order_number#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(60) Exchange +(61) CometSort 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] +Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31 ASC NULLS FIRST, cr_order_number#32 ASC NULLS FIRST] -(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 +(62) CometSortMergeJoin +Left output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +Right output [2]: [cr_item_sk#31, cr_order_number#32] +Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#31, cr_order_number#32], LeftOuter -(62) SortMergeJoin [codegen id : 7] -Left keys [2]: [cs_item_sk#4, cs_order_number#6] -Right keys [2]: [cr_item_sk#31, cr_order_number#32] -Join type: LeftOuter -Join condition: None - -(63) Project [codegen id : 7] -Output [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +(63) CometProject 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] +Arguments: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -(64) HashAggregate [codegen id : 7] +(64) CometHashAggregate 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#34] -Results [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#35] -(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] +(65) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=5] -(66) HashAggregate [codegen id : 8] -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#35] +(66) CometHashAggregate +Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] Functions [1]: [count(1)] -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] -(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] +(67) CometTakeOrderedAndProject +Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#37 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#35,promo#36,total_cnt#37]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37], 100, [total_cnt#37 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#35, promo#36, total_cnt#37] + +(68) ColumnarToRow [codegen id : 4] +Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -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 +(69) 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 -(69) CometFilter +(70) 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)) -(70) CometProject +(71) 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] -(71) ColumnarToRow [codegen id : 1] +(72) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -(72) BroadcastExchange +(73) 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=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 6cba2d0e6..1022e1a90 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,94 +1,86 @@ -TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo] - 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 (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 (4) - Sort [cs_item_sk,cs_order_number] - InputAdapter - Exchange [cs_item_sk,cs_order_number] #2 - 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] - 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 [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,d_date_sk,d_week_seq] - 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_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,d_date_sk,d_date,d_week_seq] - 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_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,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_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,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_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_sk,i_item_desc] - 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 [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,w_warehouse_sk,w_warehouse_name] - 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_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,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometFilter [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] - 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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [i_item_sk,i_item_desc] #6 - CometFilter [i_item_sk,i_item_desc] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange [cd_demo_sk] #7 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange [hd_demo_sk] #8 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange [d_date_sk,d_week_seq] #10 - CometFilter [d_date_sk,d_week_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] +WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] + CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count,count(1)] + CometColumnarExchange [i_item_desc,w_warehouse_name,d_week_seq] #1 + CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] + CometProject [w_warehouse_name,i_item_desc,d_week_seq] + CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] + CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + CometColumnarExchange [cs_item_sk,cs_order_number] #2 + RowToColumnar + 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] + 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 [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,d_date_sk,d_week_seq] + 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_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,d_date_sk,d_date,d_week_seq] + 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_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,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_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,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_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_sk,i_item_desc] + 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 [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,w_warehouse_sk,w_warehouse_name] + 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_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,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometFilter [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] + 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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange [i_item_sk,i_item_desc] #6 + CometFilter [i_item_sk,i_item_desc] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + CometBroadcastExchange [cd_demo_sk] #7 + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + CometBroadcastExchange [hd_demo_sk] #8 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] + CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + CometBroadcastExchange [d_date_sk,d_week_seq] #10 + CometFilter [d_date_sk,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter - BroadcastExchange #12 - WholeStageCodegen (2) + BroadcastExchange #11 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk] - InputAdapter - WholeStageCodegen (6) - Sort [cr_item_sk,cr_order_number] - InputAdapter - Exchange [cr_item_sk,cr_order_number] #13 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk] + CometSort [cr_item_sk,cr_order_number] + CometColumnarExchange [cr_item_sk,cr_order_number] #13 + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] 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 0ae3bf650..f63a9c4cc 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 @@ -1,75 +1,83 @@ == Physical Plan == -TakeOrderedAndProject (71) -+- * Project (70) - +- * BroadcastHashJoin Inner BuildRight (69) - :- * 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) - +- * 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) +TakeOrderedAndProject (79) ++- * Project (78) + +- * BroadcastHashJoin Inner BuildRight (77) + :- * Project (59) + : +- * BroadcastHashJoin Inner BuildRight (58) + : :- * BroadcastHashJoin Inner BuildRight (39) + : : :- * Filter (19) + : : : +- * HashAggregate (18) + : : : +- * ColumnarToRow (17) + : : : +- CometColumnarExchange (16) + : : : +- RowToColumnar (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 (38) + : : +- * HashAggregate (37) + : : +- * ColumnarToRow (36) + : : +- CometColumnarExchange (35) + : : +- RowToColumnar (34) + : : +- * HashAggregate (33) + : : +- * ColumnarToRow (32) + : : +- CometProject (31) + : : +- CometBroadcastHashJoin (30) + : : :- CometProject (26) + : : : +- CometBroadcastHashJoin (25) + : : : :- CometFilter (21) + : : : : +- CometScan parquet spark_catalog.default.customer (20) + : : : +- CometBroadcastExchange (24) + : : : +- CometFilter (23) + : : : +- CometScan parquet spark_catalog.default.store_sales (22) + : : +- CometBroadcastExchange (29) + : : +- CometFilter (28) + : : +- CometScan parquet spark_catalog.default.date_dim (27) + : +- BroadcastExchange (57) + : +- * Filter (56) + : +- * HashAggregate (55) + : +- * ColumnarToRow (54) + : +- CometColumnarExchange (53) + : +- RowToColumnar (52) + : +- * HashAggregate (51) + : +- * ColumnarToRow (50) + : +- CometProject (49) + : +- CometBroadcastHashJoin (48) + : :- CometProject (46) + : : +- CometBroadcastHashJoin (45) + : : :- CometFilter (41) + : : : +- CometScan parquet spark_catalog.default.customer (40) + : : +- CometBroadcastExchange (44) + : : +- CometFilter (43) + : : +- CometScan parquet spark_catalog.default.web_sales (42) + : +- ReusedExchange (47) + +- BroadcastExchange (76) + +- * HashAggregate (75) + +- * ColumnarToRow (74) + +- CometColumnarExchange (73) + +- RowToColumnar (72) + +- * HashAggregate (71) + +- * ColumnarToRow (70) + +- CometProject (69) + +- CometBroadcastHashJoin (68) + :- CometProject (66) + : +- CometBroadcastHashJoin (65) + : :- CometFilter (61) + : : +- CometScan parquet spark_catalog.default.customer (60) + : +- CometBroadcastExchange (64) + : +- CometFilter (63) + : +- CometScan parquet spark_catalog.default.web_sales (62) + +- ReusedExchange (67) (1) Scan parquet spark_catalog.default.customer @@ -142,33 +150,39 @@ 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] -(15) Exchange +(15) RowToColumnar 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=1] -(16) HashAggregate [codegen id : 8] +(16) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(17) ColumnarToRow [codegen id : 8] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] + +(18) 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] -(17) Filter [codegen id : 8] +(19) Filter [codegen id : 8] Input [2]: [customer_id#14, year_total#15] Condition : (isnotnull(year_total#15) AND (year_total#15 > 0.00)) -(18) Scan parquet spark_catalog.default.customer +(20) 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 -(19) CometFilter +(21) 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)) -(20) Scan parquet spark_catalog.default.store_sales +(22) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] Batched: true Location: InMemoryFileIndex [] @@ -176,90 +190,96 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(21) CometFilter +(23) CometFilter Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] Condition : isnotnull(ss_customer_sk#20) -(22) CometBroadcastExchange +(24) 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) CometBroadcastHashJoin +(25) 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) CometProject +(26) 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] -(25) Scan parquet spark_catalog.default.date_dim +(27) 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 -(26) CometFilter +(28) 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 +(29) CometBroadcastExchange Input [2]: [d_date_sk#24, d_year#25] Arguments: [d_date_sk#24, d_year#25] -(28) CometBroadcastHashJoin +(30) 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 +(31) 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] +(32) 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] -(31) HashAggregate [codegen id : 2] +(33) 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] -(32) Exchange +(34) RowToColumnar +Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] + +(35) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] + +(36) ColumnarToRow [codegen id : 3] 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=2] -(33) HashAggregate [codegen id : 3] +(37) 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] -(34) BroadcastExchange +(38) 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=3] -(35) BroadcastHashJoin [codegen id : 8] +(39) BroadcastHashJoin [codegen id : 8] Left keys [1]: [customer_id#14] Right keys [1]: [customer_id#28] Join type: Inner Join condition: None -(36) Scan parquet spark_catalog.default.customer +(40) 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 -(37) CometFilter +(41) 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)) -(38) Scan parquet spark_catalog.default.web_sales +(42) 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 [] @@ -267,86 +287,92 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(39) CometFilter +(43) CometFilter Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] Condition : isnotnull(ws_bill_customer_sk#36) -(40) CometBroadcastExchange +(44) CometBroadcastExchange Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] Arguments: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] -(41) CometBroadcastHashJoin +(45) 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) CometProject +(46) 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: 10] +(47) ReusedExchange [Reuses operator id: 10] Output [2]: [d_date_sk#40, d_year#41] -(44) CometBroadcastHashJoin +(48) 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) CometProject +(49) 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) ColumnarToRow [codegen id : 4] +(50) 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] +(51) 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] -(48) Exchange +(52) RowToColumnar +Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] + +(53) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=4] + +(54) ColumnarToRow [codegen id : 5] 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=4] -(49) HashAggregate [codegen id : 5] +(55) 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] -(50) Filter [codegen id : 5] +(56) Filter [codegen id : 5] Input [2]: [customer_id#45, year_total#46] Condition : (isnotnull(year_total#46) AND (year_total#46 > 0.00)) -(51) BroadcastExchange +(57) BroadcastExchange Input [2]: [customer_id#45, year_total#46] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(52) BroadcastHashJoin [codegen id : 8] +(58) BroadcastHashJoin [codegen id : 8] Left keys [1]: [customer_id#14] Right keys [1]: [customer_id#45] Join type: Inner Join condition: None -(53) Project [codegen id : 8] +(59) 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] -(54) Scan parquet spark_catalog.default.customer +(60) 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 -(55) CometFilter +(61) 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)) -(56) Scan parquet spark_catalog.default.web_sales +(62) 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 Location: InMemoryFileIndex [] @@ -354,128 +380,134 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(57) CometFilter +(63) CometFilter Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] Condition : isnotnull(ws_bill_customer_sk#51) -(58) CometBroadcastExchange +(64) 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) CometBroadcastHashJoin +(65) 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) CometProject +(66) 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] -(61) ReusedExchange [Reuses operator id: 27] +(67) ReusedExchange [Reuses operator id: 29] Output [2]: [d_date_sk#55, d_year#56] -(62) CometBroadcastHashJoin +(68) 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 -(63) CometProject +(69) 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] -(64) ColumnarToRow [codegen id : 6] +(70) 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] +(71) 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))] Aggregate Attributes [1]: [sum#57] Results [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] -(66) Exchange +(72) RowToColumnar +Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] + +(73) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=6] + +(74) ColumnarToRow [codegen id : 7] 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=6] -(67) HashAggregate [codegen id : 7] +(75) 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))] 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] -(68) BroadcastExchange +(76) BroadcastExchange Input [2]: [customer_id#59, year_total#60] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] -(69) BroadcastHashJoin [codegen id : 8] +(77) 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 : 8] +(78) 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] -(71) TakeOrderedAndProject +(79) 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] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (75) -+- * ColumnarToRow (74) - +- CometFilter (73) - +- CometScan parquet spark_catalog.default.date_dim (72) +BroadcastExchange (83) ++- * ColumnarToRow (82) + +- CometFilter (81) + +- CometScan parquet spark_catalog.default.date_dim (80) -(72) Scan parquet spark_catalog.default.date_dim +(80) 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 -(73) CometFilter +(81) 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)) -(74) ColumnarToRow [codegen id : 1] +(82) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_year#10] -(75) BroadcastExchange +(83) BroadcastExchange Input [2]: [d_date_sk#9, d_year#10] 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) -+- * ColumnarToRow (78) - +- CometFilter (77) - +- CometScan parquet spark_catalog.default.date_dim (76) +Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 +BroadcastExchange (87) ++- * ColumnarToRow (86) + +- CometFilter (85) + +- CometScan parquet spark_catalog.default.date_dim (84) -(76) Scan parquet spark_catalog.default.date_dim +(84) 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 -(77) CometFilter +(85) 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)) -(78) ColumnarToRow [codegen id : 1] +(86) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#24, d_year#25] -(79) BroadcastExchange +(87) BroadcastExchange Input [2]: [d_date_sk#24, d_year#25] 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#38 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 42 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 +Subquery:4 Hosting operator id = 62 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 4360a9555..cf84139a1 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 @@ -7,38 +7,11 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] BroadcastHashJoin [customer_id,customer_id] Filter [year_total] 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 (1) - 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 [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_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_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - 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 + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 + RowToColumnar + WholeStageCodegen (1) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] ColumnarToRow InputAdapter @@ -48,59 +21,94 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 + CometBroadcastExchange [d_date_sk,d_year] #4 CometFilter [d_date_sk,d_year] 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #6 + RowToColumnar + 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 [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter 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] #11 - WholeStageCodegen (4) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 - CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_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 + ColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #11 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_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 #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] #14 - WholeStageCodegen (6) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 - CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_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 + ColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #14 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_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 d45346b6c..bcf6fc534 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,136 +1,137 @@ == Physical Plan == -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) +TakeOrderedAndProject (133) ++- * Project (132) + +- * SortMergeJoin Inner (131) + :- * ColumnarToRow (71) + : +- CometSort (70) + : +- CometColumnarExchange (69) + : +- RowToColumnar (68) + : +- * Filter (67) + : +- * HashAggregate (66) + : +- * ColumnarToRow (65) + : +- CometColumnarExchange (64) + : +- RowToColumnar (63) + : +- * HashAggregate (62) + : +- * ColumnarToRow (61) + : +- CometHashAggregate (60) + : +- CometColumnarExchange (59) + : +- CometHashAggregate (58) + : +- CometUnion (57) + : :- CometProject (22) + : : +- CometSortMergeJoin (21) + : : :- CometSort (15) + : : : +- CometColumnarExchange (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) + : : +- CometSort (20) + : : +- CometColumnarExchange (19) + : : +- CometProject (18) + : : +- CometFilter (17) + : : +- CometScan parquet spark_catalog.default.catalog_returns (16) + : :- CometProject (39) + : : +- CometSortMergeJoin (38) + : : :- CometSort (32) + : : : +- CometColumnarExchange (31) + : : : +- CometProject (30) + : : : +- CometBroadcastHashJoin (29) + : : : :- CometProject (27) + : : : : +- CometBroadcastHashJoin (26) + : : : : :- CometFilter (24) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (23) + : : : : +- ReusedExchange (25) + : : : +- ReusedExchange (28) + : : +- CometSort (37) + : : +- CometColumnarExchange (36) + : : +- CometProject (35) + : : +- CometFilter (34) + : : +- CometScan parquet spark_catalog.default.store_returns (33) + : +- CometProject (56) + : +- CometSortMergeJoin (55) + : :- CometSort (49) + : : +- CometColumnarExchange (48) + : : +- CometProject (47) + : : +- CometBroadcastHashJoin (46) + : : :- CometProject (44) + : : : +- CometBroadcastHashJoin (43) + : : : :- CometFilter (41) + : : : : +- CometScan parquet spark_catalog.default.web_sales (40) + : : : +- ReusedExchange (42) + : : +- ReusedExchange (45) + : +- CometSort (54) + : +- CometColumnarExchange (53) + : +- CometProject (52) + : +- CometFilter (51) + : +- CometScan parquet spark_catalog.default.web_returns (50) + +- * ColumnarToRow (130) + +- CometSort (129) + +- CometColumnarExchange (128) + +- RowToColumnar (127) + +- * Filter (126) + +- * HashAggregate (125) + +- * ColumnarToRow (124) + +- CometColumnarExchange (123) + +- RowToColumnar (122) + +- * HashAggregate (121) + +- * ColumnarToRow (120) + +- CometHashAggregate (119) + +- CometColumnarExchange (118) + +- CometHashAggregate (117) + +- CometUnion (116) + :- CometProject (87) + : +- CometSortMergeJoin (86) + : :- CometSort (83) + : : +- CometColumnarExchange (82) + : : +- CometProject (81) + : : +- CometBroadcastHashJoin (80) + : : :- CometProject (76) + : : : +- CometBroadcastHashJoin (75) + : : : :- CometFilter (73) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (72) + : : : +- ReusedExchange (74) + : : +- CometBroadcastExchange (79) + : : +- CometFilter (78) + : : +- CometScan parquet spark_catalog.default.date_dim (77) + : +- CometSort (85) + : +- ReusedExchange (84) + :- CometProject (101) + : +- CometSortMergeJoin (100) + : :- CometSort (97) + : : +- CometColumnarExchange (96) + : : +- CometProject (95) + : : +- CometBroadcastHashJoin (94) + : : :- CometProject (92) + : : : +- CometBroadcastHashJoin (91) + : : : :- CometFilter (89) + : : : : +- CometScan parquet spark_catalog.default.store_sales (88) + : : : +- ReusedExchange (90) + : : +- ReusedExchange (93) + : +- CometSort (99) + : +- ReusedExchange (98) + +- CometProject (115) + +- CometSortMergeJoin (114) + :- CometSort (111) + : +- CometColumnarExchange (110) + : +- CometProject (109) + : +- CometBroadcastHashJoin (108) + : :- CometProject (106) + : : +- CometBroadcastHashJoin (105) + : : :- CometFilter (103) + : : : +- CometScan parquet spark_catalog.default.web_sales (102) + : : +- ReusedExchange (104) + : +- ReusedExchange (107) + +- CometSort (113) + +- ReusedExchange (112) (1) Scan parquet spark_catalog.default.catalog_sales @@ -197,54 +198,47 @@ Arguments: [cs_sold_date_sk#5], [d_date_sk#13], Inner, BuildRight 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] +(14) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(15) Exchange +(15) CometSort 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=1] +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_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST] -(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 - -(17) Scan parquet spark_catalog.default.catalog_returns +(16) 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 -(18) CometFilter +(17) 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)) -(19) CometProject +(18) 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] -(20) ColumnarToRow [codegen id : 3] -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] - -(21) Exchange +(19) CometColumnarExchange 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=2] +Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(22) Sort [codegen id : 4] +(20) CometSort 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 +Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST] -(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 +(21) CometSortMergeJoin +Left 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] +Right output [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] +Arguments: [cs_order_number#2, cs_item_sk#1], [cr_order_number#16, cr_item_sk#15], LeftOuter -(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] +(22) CometProject 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] +Arguments: [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], [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] -(25) Scan parquet spark_catalog.default.store_sales +(23) 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 [] @@ -252,82 +246,75 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(26) CometFilter +(24) 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) -(27) ReusedExchange [Reuses operator id: 6] +(25) 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) CometBroadcastHashJoin +(26) 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) CometProject +(27) 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: 11] +(28) ReusedExchange [Reuses operator id: 11] Output [2]: [d_date_sk#33, d_year#34] -(31) CometBroadcastHashJoin +(29) 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) CometProject +(30) 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] - -(34) Exchange +(31) CometColumnarExchange 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=3] +Arguments: hashpartitioning(ss_ticket_number#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(35) Sort [codegen id : 7] +(32) CometSort 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 +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_ticket_number#23 ASC NULLS FIRST, ss_item_sk#22 ASC NULLS FIRST] -(36) Scan parquet spark_catalog.default.store_returns +(33) 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 -(37) CometFilter +(34) 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)) -(38) CometProject +(35) 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] -(39) ColumnarToRow [codegen id : 8] +(36) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=4] -(40) Exchange +(37) CometSort 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=4] +Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST] -(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 - -(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 +(38) CometSortMergeJoin +Left 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] +Right output [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] +Arguments: [ss_ticket_number#23, ss_item_sk#22], [sr_ticket_number#36, sr_item_sk#35], LeftOuter -(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] +(39) CometProject 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] +Arguments: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41], [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] -(44) Scan parquet spark_catalog.default.web_sales +(40) 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 [] @@ -335,132 +322,139 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(45) CometFilter +(41) 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) -(46) ReusedExchange [Reuses operator id: 6] +(42) 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) CometBroadcastHashJoin +(43) 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) CometProject +(44) 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: 11] +(45) ReusedExchange [Reuses operator id: 11] Output [2]: [d_date_sk#53, d_year#54] -(50) CometBroadcastHashJoin +(46) 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) CometProject +(47) 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) ColumnarToRow [codegen id : 11] +(48) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=5] -(53) Exchange +(49) CometSort 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] +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_order_number#43 ASC NULLS FIRST, ws_item_sk#42 ASC NULLS FIRST] -(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 - -(55) Scan parquet spark_catalog.default.web_returns +(50) 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 -(56) CometFilter +(51) 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)) -(57) CometProject +(52) 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] -(58) ColumnarToRow [codegen id : 13] +(53) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=6] -(59) Exchange +(54) CometSort 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=6] +Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST] -(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 +(55) CometSortMergeJoin +Left 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] +Right output [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] +Arguments: [ws_order_number#43, ws_item_sk#42], [wr_order_number#56, wr_item_sk#55], LeftOuter -(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 - -(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] +(56) CometProject 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] +Arguments: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61], [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] -(63) Union +(57) CometUnion +Child 0 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] +Child 1 Input [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41] +Child 2 Input [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61] -(64) HashAggregate [codegen id : 16] +(58) CometHashAggregate 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] -(65) Exchange +(59) CometColumnarExchange 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=7] +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, CometColumnarShuffle, [plan_id=7] -(66) HashAggregate [codegen id : 17] +(60) CometHashAggregate 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] -(67) HashAggregate [codegen id : 17] +(61) ColumnarToRow [codegen id : 1] +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] + +(62) HashAggregate [codegen id : 1] 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] -(68) Exchange +(63) RowToColumnar +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] + +(64) CometColumnarExchange 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=8] +Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(69) HashAggregate [codegen id : 18] +(65) ColumnarToRow [codegen id : 2] +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] + +(66) HashAggregate [codegen id : 2] 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] -(70) Filter [codegen id : 18] +(67) Filter [codegen id : 2] 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) -(71) Exchange +(68) RowToColumnar +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] + +(69) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=9] + +(70) CometSort 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=9] +Arguments: [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], [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] -(72) Sort [codegen id : 19] +(71) ColumnarToRow [codegen id : 3] 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 -(73) Scan parquet spark_catalog.default.catalog_sales +(72) 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 [] @@ -468,75 +462,71 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#74), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(74) CometFilter +(73) 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) -(75) ReusedExchange [Reuses operator id: 6] +(74) 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) CometBroadcastHashJoin +(75) 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) CometProject +(76) 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) Scan parquet spark_catalog.default.date_dim +(77) 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) CometFilter +(78) 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) CometBroadcastExchange +(79) CometBroadcastExchange Input [2]: [d_date_sk#81, d_year#82] Arguments: [d_date_sk#81, d_year#82] -(81) CometBroadcastHashJoin +(80) 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 +(81) 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] - -(84) Exchange +(82) CometColumnarExchange 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=10] +Arguments: hashpartitioning(cs_order_number#71, cs_item_sk#70, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(85) Sort [codegen id : 21] +(83) CometSort 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 +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_order_number#71 ASC NULLS FIRST, cs_item_sk#70 ASC NULLS FIRST] -(86) ReusedExchange [Reuses operator id: 21] +(84) ReusedExchange [Reuses operator id: 19] Output [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] -(87) Sort [codegen id : 23] +(85) CometSort 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 +Arguments: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86], [cr_order_number#84 ASC NULLS FIRST, cr_item_sk#83 ASC NULLS FIRST] -(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) CometSortMergeJoin +Left 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] +Right output [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] +Arguments: [cs_order_number#71, cs_item_sk#70], [cr_order_number#84, cr_item_sk#83], LeftOuter -(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] +(87) CometProject 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] +Arguments: [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], [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] -(90) Scan parquet spark_catalog.default.store_sales +(88) 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 [] @@ -544,63 +534,59 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#93), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(91) CometFilter +(89) 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) -(92) ReusedExchange [Reuses operator id: 6] +(90) 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] -(93) CometBroadcastHashJoin +(91) 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 -(94) CometProject +(92) 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] -(95) ReusedExchange [Reuses operator id: 80] +(93) ReusedExchange [Reuses operator id: 79] Output [2]: [d_date_sk#100, d_year#101] -(96) CometBroadcastHashJoin +(94) 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 -(97) CometProject +(95) 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] -(98) ColumnarToRow [codegen id : 25] +(96) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=11] -(99) Exchange +(97) CometSort 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] +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_ticket_number#90 ASC NULLS FIRST, ss_item_sk#89 ASC NULLS FIRST] -(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 - -(101) ReusedExchange [Reuses operator id: 40] +(98) ReusedExchange [Reuses operator id: 36] Output [4]: [sr_item_sk#102, sr_ticket_number#103, sr_return_quantity#104, sr_return_amt#105] -(102) Sort [codegen id : 28] +(99) CometSort 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 +Arguments: [sr_item_sk#102, sr_ticket_number#103, sr_return_quantity#104, sr_return_amt#105], [sr_ticket_number#103 ASC NULLS FIRST, sr_item_sk#102 ASC NULLS FIRST] -(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 +(100) CometSortMergeJoin +Left 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] +Right output [4]: [sr_item_sk#102, sr_ticket_number#103, sr_return_quantity#104, sr_return_amt#105] +Arguments: [ss_ticket_number#90, ss_item_sk#89], [sr_ticket_number#103, sr_item_sk#102], LeftOuter -(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] +(101) CometProject 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] +Arguments: [d_year#101, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99, sales_cnt#106, sales_amt#107], [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] -(105) Scan parquet spark_catalog.default.web_sales +(102) 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 [] @@ -608,184 +594,194 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#112), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(106) CometFilter +(103) 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) -(107) ReusedExchange [Reuses operator id: 6] +(104) 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] -(108) CometBroadcastHashJoin +(105) 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 -(109) CometProject +(106) 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] -(110) ReusedExchange [Reuses operator id: 80] +(107) ReusedExchange [Reuses operator id: 79] Output [2]: [d_date_sk#119, d_year#120] -(111) CometBroadcastHashJoin +(108) 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 -(112) CometProject +(109) 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] - -(114) Exchange +(110) CometColumnarExchange 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=12] +Arguments: hashpartitioning(ws_order_number#109, ws_item_sk#108, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(115) Sort [codegen id : 31] +(111) CometSort 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 +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_order_number#109 ASC NULLS FIRST, ws_item_sk#108 ASC NULLS FIRST] -(116) ReusedExchange [Reuses operator id: 59] +(112) ReusedExchange [Reuses operator id: 53] Output [4]: [wr_item_sk#121, wr_order_number#122, wr_return_quantity#123, wr_return_amt#124] -(117) Sort [codegen id : 33] +(113) CometSort 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 +Arguments: [wr_item_sk#121, wr_order_number#122, wr_return_quantity#123, wr_return_amt#124], [wr_order_number#122 ASC NULLS FIRST, wr_item_sk#121 ASC NULLS FIRST] -(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 +(114) CometSortMergeJoin +Left 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] +Right output [4]: [wr_item_sk#121, wr_order_number#122, wr_return_quantity#123, wr_return_amt#124] +Arguments: [ws_order_number#109, ws_item_sk#108], [wr_order_number#122, wr_item_sk#121], LeftOuter -(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] +(115) CometProject 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] +Arguments: [d_year#120, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118, sales_cnt#125, sales_amt#126], [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] -(120) Union +(116) CometUnion +Child 0 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] +Child 1 Input [7]: [d_year#101, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99, sales_cnt#106, sales_amt#107] +Child 2 Input [7]: [d_year#120, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118, sales_cnt#125, sales_amt#126] -(121) HashAggregate [codegen id : 35] +(117) CometHashAggregate 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] -(122) Exchange +(118) CometColumnarExchange 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=13] +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, CometColumnarShuffle, [plan_id=13] -(123) HashAggregate [codegen id : 36] +(119) CometHashAggregate 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] -(124) HashAggregate [codegen id : 36] +(120) ColumnarToRow [codegen id : 4] +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] + +(121) HashAggregate [codegen id : 4] 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] -(125) Exchange +(122) RowToColumnar +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] + +(123) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=14] + +(124) ColumnarToRow [codegen id : 5] 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=14] -(126) HashAggregate [codegen id : 37] +(125) HashAggregate [codegen id : 5] 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] -(127) Filter [codegen id : 37] +(126) Filter [codegen id : 5] 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) -(128) Exchange +(127) RowToColumnar +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] + +(128) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=15] + +(129) CometSort 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=15] +Arguments: [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], [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] -(129) Sort [codegen id : 38] +(130) ColumnarToRow [codegen id : 6] 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 -(130) SortMergeJoin [codegen id : 39] +(131) SortMergeJoin [codegen id : 7] 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) -(131) Project [codegen id : 39] +(132) Project [codegen id : 7] 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] -(132) TakeOrderedAndProject +(133) 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 (136) -+- * ColumnarToRow (135) - +- CometFilter (134) - +- CometScan parquet spark_catalog.default.date_dim (133) +BroadcastExchange (137) ++- * ColumnarToRow (136) + +- CometFilter (135) + +- CometScan parquet spark_catalog.default.date_dim (134) -(133) Scan parquet spark_catalog.default.date_dim +(134) 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 -(134) CometFilter +(135) 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)) -(135) ColumnarToRow [codegen id : 1] +(136) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] -(136) BroadcastExchange +(137) BroadcastExchange Input [2]: [d_date_sk#13, d_year#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 -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) +Subquery:4 Hosting operator id = 72 Hosting Expression = cs_sold_date_sk#74 IN dynamicpruning#75 +BroadcastExchange (141) ++- * ColumnarToRow (140) + +- CometFilter (139) + +- CometScan parquet spark_catalog.default.date_dim (138) -(137) Scan parquet spark_catalog.default.date_dim +(138) 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 -(138) CometFilter +(139) 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)) -(139) ColumnarToRow [codegen id : 1] +(140) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#81, d_year#82] -(140) BroadcastExchange +(141) BroadcastExchange Input [2]: [d_date_sk#81, d_year#82] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] -Subquery:5 Hosting operator id = 90 Hosting Expression = ss_sold_date_sk#93 IN dynamicpruning#75 +Subquery:5 Hosting operator id = 88 Hosting Expression = ss_sold_date_sk#93 IN dynamicpruning#75 -Subquery:6 Hosting operator id = 105 Hosting Expression = ws_sold_date_sk#112 IN dynamicpruning#75 +Subquery:6 Hosting operator id = 102 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 2aecc9e98..6fb16b648 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,229 +1,166 @@ 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 (39) + WholeStageCodegen (7) 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 (19) - Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] + WholeStageCodegen (3) + ColumnarToRow InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - 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 (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 (16) - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 + RowToColumnar + WholeStageCodegen (2) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] + ColumnarToRow InputAdapter - Union - 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 (2) - Sort [cs_order_number,cs_item_sk] - InputAdapter - Exchange [cs_order_number,cs_item_sk] #4 - 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_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,d_date_sk,d_year] - 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,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_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 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,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 [d_date_sk,d_year] #7 - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - WholeStageCodegen (4) - Sort [cr_order_number,cr_item_sk] - InputAdapter - 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_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_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 (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 (7) - Sort [ss_ticket_number,ss_item_sk] - 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_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,d_date_sk,d_year] - 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,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_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 - 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 (9) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - 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_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_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 (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 (12) - Sort [ws_order_number,ws_item_sk] - 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_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,d_date_sk,d_year] - 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,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_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 - 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 (14) - Sort [wr_order_number,wr_item_sk] - InputAdapter - 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_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometSort [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] + CometColumnarExchange [cs_order_number,cs_item_sk] #4 + 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_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,d_date_sk,d_year] + 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,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_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 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,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 [d_date_sk,d_year] #7 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometColumnarExchange [cr_order_number,cr_item_sk] #8 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometSort [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] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #9 + 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_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,d_date_sk,d_year] + 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,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_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 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #7 + CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometColumnarExchange [sr_ticket_number,sr_item_sk] #10 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometSort [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] + CometColumnarExchange [ws_order_number,ws_item_sk] #11 + 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_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,d_date_sk,d_year] + 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,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_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 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #7 + CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometColumnarExchange [wr_order_number,wr_item_sk] #12 + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_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 (38) - Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] + WholeStageCodegen (6) + ColumnarToRow InputAdapter - 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] #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] #15 - WholeStageCodegen (35) - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 + RowToColumnar + WholeStageCodegen (5) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] + ColumnarToRow InputAdapter - Union - 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 (21) - Sort [cs_order_number,cs_item_sk] - 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_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,d_date_sk,d_year] - 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,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_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 #17 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometBroadcastExchange [d_date_sk,d_year] #18 - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - WholeStageCodegen (23) - Sort [cr_order_number,cr_item_sk] - InputAdapter - 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 (26) - Sort [ss_ticket_number,ss_item_sk] - 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_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,d_date_sk,d_year] - 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,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_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 - 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 (28) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - 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 (31) - Sort [ws_order_number,ws_item_sk] - 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_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,d_date_sk,d_year] - 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,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_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 - 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 (33) - Sort [wr_order_number,wr_item_sk] - InputAdapter - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #12 + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometSort [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] + CometColumnarExchange [cs_order_number,cs_item_sk] #16 + 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_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,d_date_sk,d_year] + 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,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_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 #17 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + CometBroadcastExchange [d_date_sk,d_year] #18 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #8 + CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometSort [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] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #19 + 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_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,d_date_sk,d_year] + 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,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_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 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #18 + CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #10 + CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometSort [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] + CometColumnarExchange [ws_order_number,ws_item_sk] #20 + 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_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,d_date_sk,d_year] + 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,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_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 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #18 + CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + 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 b72c8bebb..b2c2d2797 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,103 +1,125 @@ == Physical Plan == -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) +* ColumnarToRow (121) ++- CometTakeOrderedAndProject (120) + +- CometHashAggregate (119) + +- CometColumnarExchange (118) + +- RowToColumnar (117) + +- * HashAggregate (116) + +- Union (115) + :- * HashAggregate (98) + : +- * ColumnarToRow (97) + : +- CometColumnarExchange (96) + : +- RowToColumnar (95) + : +- * HashAggregate (94) + : +- Union (93) + : :- * Project (36) + : : +- * BroadcastHashJoin LeftOuter BuildRight (35) + : : :- * HashAggregate (19) + : : : +- * ColumnarToRow (18) + : : : +- CometColumnarExchange (17) + : : : +- RowToColumnar (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 (34) + : : +- * HashAggregate (33) + : : +- * ColumnarToRow (32) + : : +- CometColumnarExchange (31) + : : +- RowToColumnar (30) + : : +- * HashAggregate (29) + : : +- * ColumnarToRow (28) + : : +- CometProject (27) + : : +- CometBroadcastHashJoin (26) + : : :- CometProject (24) + : : : +- CometBroadcastHashJoin (23) + : : : :- CometFilter (21) + : : : : +- CometScan parquet spark_catalog.default.store_returns (20) + : : : +- ReusedExchange (22) + : : +- ReusedExchange (25) + : :- * Project (59) + : : +- * BroadcastNestedLoopJoin Inner BuildLeft (58) + : : :- BroadcastExchange (47) + : : : +- * HashAggregate (46) + : : : +- * ColumnarToRow (45) + : : : +- CometColumnarExchange (44) + : : : +- RowToColumnar (43) + : : : +- * HashAggregate (42) + : : : +- * ColumnarToRow (41) + : : : +- CometProject (40) + : : : +- CometBroadcastHashJoin (39) + : : : :- CometScan parquet spark_catalog.default.catalog_sales (37) + : : : +- ReusedExchange (38) + : : +- * HashAggregate (57) + : : +- * ColumnarToRow (56) + : : +- CometColumnarExchange (55) + : : +- RowToColumnar (54) + : : +- * HashAggregate (53) + : : +- * ColumnarToRow (52) + : : +- CometProject (51) + : : +- CometBroadcastHashJoin (50) + : : :- CometScan parquet spark_catalog.default.catalog_returns (48) + : : +- ReusedExchange (49) + : +- * Project (92) + : +- * BroadcastHashJoin LeftOuter BuildRight (91) + : :- * HashAggregate (75) + : : +- * ColumnarToRow (74) + : : +- CometColumnarExchange (73) + : : +- RowToColumnar (72) + : : +- * HashAggregate (71) + : : +- * ColumnarToRow (70) + : : +- CometProject (69) + : : +- CometBroadcastHashJoin (68) + : : :- CometProject (64) + : : : +- CometBroadcastHashJoin (63) + : : : :- CometFilter (61) + : : : : +- CometScan parquet spark_catalog.default.web_sales (60) + : : : +- ReusedExchange (62) + : : +- CometBroadcastExchange (67) + : : +- CometFilter (66) + : : +- CometScan parquet spark_catalog.default.web_page (65) + : +- BroadcastExchange (90) + : +- * HashAggregate (89) + : +- * ColumnarToRow (88) + : +- CometColumnarExchange (87) + : +- RowToColumnar (86) + : +- * HashAggregate (85) + : +- * ColumnarToRow (84) + : +- CometProject (83) + : +- CometBroadcastHashJoin (82) + : :- CometProject (80) + : : +- CometBroadcastHashJoin (79) + : : :- CometFilter (77) + : : : +- CometScan parquet spark_catalog.default.web_returns (76) + : : +- ReusedExchange (78) + : +- ReusedExchange (81) + :- * HashAggregate (106) + : +- * ColumnarToRow (105) + : +- CometColumnarExchange (104) + : +- RowToColumnar (103) + : +- * HashAggregate (102) + : +- * HashAggregate (101) + : +- * ColumnarToRow (100) + : +- ReusedExchange (99) + +- * HashAggregate (114) + +- * ColumnarToRow (113) + +- CometColumnarExchange (112) + +- RowToColumnar (111) + +- * HashAggregate (110) + +- * HashAggregate (109) + +- * ColumnarToRow (108) + +- ReusedExchange (107) (1) Scan parquet spark_catalog.default.store_sales @@ -174,18 +196,24 @@ Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(Un Aggregate Attributes [2]: [sum#9, sum#10] Results [3]: [s_store_sk#8, sum#11, sum#12] -(16) Exchange +(16) RowToColumnar Input [3]: [s_store_sk#8, sum#11, sum#12] -Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(17) HashAggregate [codegen id : 4] +(17) CometColumnarExchange +Input [3]: [s_store_sk#8, sum#11, sum#12] +Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(18) ColumnarToRow [codegen id : 4] +Input [3]: [s_store_sk#8, sum#11, sum#12] + +(19) 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))#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] -(18) Scan parquet spark_catalog.default.store_returns +(20) 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 [] @@ -193,162 +221,180 @@ PartitionFilters: [isnotnull(sr_returned_date_sk#20), dynamicpruningexpression(s PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(19) CometFilter +(21) 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] +(22) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#22] -(21) CometBroadcastHashJoin +(23) 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 +(24) 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] +(25) ReusedExchange [Reuses operator id: 11] Output [1]: [s_store_sk#23] -(24) CometBroadcastHashJoin +(26) 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 +(27) 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] +(28) ColumnarToRow [codegen id : 2] Input [3]: [sr_return_amt#18, sr_net_loss#19, s_store_sk#23] -(27) HashAggregate [codegen id : 2] +(29) 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 +(30) RowToColumnar +Input [3]: [s_store_sk#23, sum#26, sum#27] + +(31) CometColumnarExchange Input [3]: [s_store_sk#23, sum#26, sum#27] -Arguments: hashpartitioning(s_store_sk#23, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(s_store_sk#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(29) HashAggregate [codegen id : 3] +(32) ColumnarToRow [codegen id : 3] +Input [3]: [s_store_sk#23, sum#26, sum#27] + +(33) 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 +(34) 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] +(35) BroadcastHashJoin [codegen id : 4] Left keys [1]: [s_store_sk#8] Right keys [1]: [s_store_sk#23] Join type: LeftOuter Join condition: None -(32) Project [codegen id : 4] +(36) 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] -(33) Scan parquet spark_catalog.default.catalog_sales +(37) 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#39), dynamicpruningexpression(cs_sold_date_sk#39 IN dynamicpruning#40)] ReadSchema: struct -(34) ReusedExchange [Reuses operator id: 6] +(38) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#41] -(35) CometBroadcastHashJoin +(39) 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 +(40) 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] +(41) 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] +(42) 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 +(43) RowToColumnar +Input [3]: [cs_call_center_sk#36, sum#44, sum#45] + +(44) CometColumnarExchange +Input [3]: [cs_call_center_sk#36, sum#44, sum#45] +Arguments: hashpartitioning(cs_call_center_sk#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(45) ColumnarToRow [codegen id : 6] 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] +(46) 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 +(47) 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 +(48) 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#52), dynamicpruningexpression(cr_returned_date_sk#52 IN dynamicpruning#53)] ReadSchema: struct -(43) ReusedExchange [Reuses operator id: 6] +(49) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#54] -(44) CometBroadcastHashJoin +(50) 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 -(45) CometProject +(51) 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] -(46) ColumnarToRow [codegen id : 7] +(52) ColumnarToRow [codegen id : 7] Input [2]: [cr_return_amount#50, cr_net_loss#51] -(47) HashAggregate [codegen id : 7] +(53) HashAggregate [codegen id : 7] Input [2]: [cr_return_amount#50, cr_net_loss#51] Keys: [] 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] -(48) Exchange +(54) RowToColumnar Input [2]: [sum#57, sum#58] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(49) HashAggregate +(55) CometColumnarExchange +Input [2]: [sum#57, sum#58] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(56) ColumnarToRow +Input [2]: [sum#57, sum#58] + +(57) HashAggregate Input [2]: [sum#57, sum#58] Keys: [] 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] -(50) BroadcastNestedLoopJoin [codegen id : 8] +(58) BroadcastNestedLoopJoin [codegen id : 8] Join type: Inner Join condition: None -(51) Project [codegen id : 8] +(59) 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] -(52) Scan parquet spark_catalog.default.web_sales +(60) 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 [] @@ -356,68 +402,74 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#69), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct -(53) CometFilter +(61) 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) -(54) ReusedExchange [Reuses operator id: 6] +(62) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#71] -(55) CometBroadcastHashJoin +(63) 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 -(56) CometProject +(64) 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] -(57) Scan parquet spark_catalog.default.web_page +(65) 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 -(58) CometFilter +(66) CometFilter Input [1]: [wp_web_page_sk#72] Condition : isnotnull(wp_web_page_sk#72) -(59) CometBroadcastExchange +(67) CometBroadcastExchange Input [1]: [wp_web_page_sk#72] Arguments: [wp_web_page_sk#72] -(60) CometBroadcastHashJoin +(68) 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 +(69) 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] +(70) 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] +(71) 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 +(72) RowToColumnar +Input [3]: [wp_web_page_sk#72, sum#75, sum#76] + +(73) CometColumnarExchange Input [3]: [wp_web_page_sk#72, sum#75, sum#76] -Arguments: hashpartitioning(wp_web_page_sk#72, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Arguments: hashpartitioning(wp_web_page_sk#72, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(65) HashAggregate [codegen id : 12] +(74) ColumnarToRow [codegen id : 12] +Input [3]: [wp_web_page_sk#72, sum#75, sum#76] + +(75) 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 +(76) 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 [] @@ -425,209 +477,243 @@ PartitionFilters: [isnotnull(wr_returned_date_sk#84), dynamicpruningexpression(w PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct -(67) CometFilter +(77) 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: 6] +(78) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#86] -(69) CometBroadcastHashJoin +(79) 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) CometProject +(80) 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] +(81) ReusedExchange [Reuses operator id: 67] Output [1]: [wp_web_page_sk#87] -(72) CometBroadcastHashJoin +(82) 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 +(83) 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] +(84) ColumnarToRow [codegen id : 10] Input [3]: [wr_return_amt#82, wr_net_loss#83, wp_web_page_sk#87] -(75) HashAggregate [codegen id : 10] +(85) 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 +(86) RowToColumnar +Input [3]: [wp_web_page_sk#87, sum#90, sum#91] + +(87) CometColumnarExchange +Input [3]: [wp_web_page_sk#87, sum#90, sum#91] +Arguments: hashpartitioning(wp_web_page_sk#87, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(88) ColumnarToRow [codegen id : 11] 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] +(89) 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 +(90) 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] +(91) 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 -(80) Project [codegen id : 12] +(92) 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 +(93) Union -(82) HashAggregate [codegen id : 13] +(94) 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 +(95) RowToColumnar +Input [8]: [channel#32, id#33, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] + +(96) CometColumnarExchange 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] +Arguments: hashpartitioning(channel#32, id#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(84) HashAggregate [codegen id : 14] +(97) ColumnarToRow [codegen id : 14] +Input [8]: [channel#32, id#33, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] + +(98) 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] +(99) ReusedExchange [Reuses operator id: 96] Output [8]: [channel#118, id#119, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] -(86) HashAggregate [codegen id : 28] +(100) ColumnarToRow [codegen id : 28] +Input [8]: [channel#118, id#119, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] + +(101) 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] +(102) 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 +(103) RowToColumnar +Input [7]: [channel#118, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] + +(104) CometColumnarExchange 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] +Arguments: hashpartitioning(channel#118, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(89) HashAggregate [codegen id : 29] +(105) ColumnarToRow [codegen id : 29] +Input [7]: [channel#118, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] + +(106) 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] +(107) ReusedExchange [Reuses operator id: 96] Output [8]: [channel#151, id#152, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] -(91) HashAggregate [codegen id : 43] +(108) ColumnarToRow [codegen id : 43] +Input [8]: [channel#151, id#152, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] + +(109) 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] +(110) HashAggregate [codegen id : 43] Input [3]: [sales#162, returns#163, profit#164] Keys: [] 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] -(93) Exchange +(111) RowToColumnar +Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] + +(112) CometColumnarExchange Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(94) HashAggregate [codegen id : 44] +(113) ColumnarToRow [codegen id : 44] +Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] + +(114) HashAggregate [codegen id : 44] Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] Keys: [] 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] -(95) Union +(115) Union -(96) HashAggregate [codegen id : 45] +(116) 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#32, id#33, sales#115, returns#116, profit#117] -(97) Exchange +(117) RowToColumnar 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] -(98) HashAggregate [codegen id : 46] +(118) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=13] + +(119) CometHashAggregate 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#32, id#33, sales#115, returns#116, profit#117] -(99) TakeOrderedAndProject +(120) CometTakeOrderedAndProject +Input [5]: [channel#32, id#33, sales#115, returns#116, profit#117] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#32 ASC NULLS FIRST,id#33 ASC NULLS FIRST], output=[channel#32,id#33,sales#115,returns#116,profit#117]), [channel#32, id#33, sales#115, returns#116, profit#117], 100, [channel#32 ASC NULLS FIRST, id#33 ASC NULLS FIRST], [channel#32, id#33, sales#115, returns#116, profit#117] + +(121) ColumnarToRow [codegen id : 46] 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 (104) -+- * ColumnarToRow (103) - +- CometProject (102) - +- CometFilter (101) - +- CometScan parquet spark_catalog.default.date_dim (100) +BroadcastExchange (126) ++- * ColumnarToRow (125) + +- CometProject (124) + +- CometFilter (123) + +- CometScan parquet spark_catalog.default.date_dim (122) -(100) Scan parquet spark_catalog.default.date_dim +(122) 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 -(101) CometFilter +(123) 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)) -(102) CometProject +(124) CometProject Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(103) ColumnarToRow [codegen id : 1] +(125) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(104) BroadcastExchange +(126) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] -Subquery:2 Hosting operator id = 18 Hosting Expression = sr_returned_date_sk#20 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 20 Hosting Expression = sr_returned_date_sk#20 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#39 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 37 Hosting Expression = cs_sold_date_sk#39 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 42 Hosting Expression = cr_returned_date_sk#52 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 48 Hosting Expression = cr_returned_date_sk#52 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 52 Hosting Expression = ws_sold_date_sk#69 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#69 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 66 Hosting Expression = wr_returned_date_sk#84 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 76 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 db4e10247..df6c4bc88 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,155 +1,177 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (46) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Exchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (45) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - 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 (13) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - 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 (1) - HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] +WholeStageCodegen (46) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [channel,id,sales,returns,profit] + CometColumnarExchange [channel,id,sales,returns,profit] #1 + RowToColumnar + WholeStageCodegen (45) + HashAggregate [channel,id,sales,returns,profit] + InputAdapter + Union + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel,id] #2 + RowToColumnar + WholeStageCodegen (13) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + 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] ColumnarToRow InputAdapter - CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] - CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_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 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + CometColumnarExchange [s_store_sk] #3 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] + ColumnarToRow + InputAdapter + CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_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 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] - InputAdapter - 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] + CometBroadcastExchange [s_store_sk] #6 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] InputAdapter - Exchange [s_store_sk] #8 - WholeStageCodegen (2) - HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] + 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] ColumnarToRow InputAdapter - CometProject [sr_return_amt,sr_net_loss,s_store_sk] - CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] - CometProject [sr_store_sk,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_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 - 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 #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] + CometColumnarExchange [s_store_sk] #8 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] + ColumnarToRow + InputAdapter + CometProject [sr_return_amt,sr_net_loss,s_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] + CometProject [sr_store_sk,sr_return_amt,sr_net_loss] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_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 + ReusedExchange [d_date_sk] #5 + ReusedExchange [s_store_sk] #6 + WholeStageCodegen (8) + Project [cs_call_center_sk,sales,returns,profit,profit_loss] + BroadcastNestedLoopJoin InputAdapter - Exchange [cs_call_center_sk] #10 - WholeStageCodegen (5) - HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] + 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] ColumnarToRow InputAdapter - CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,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 - 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 #11 - WholeStageCodegen (7) - HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] + CometColumnarExchange [cs_call_center_sk] #10 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] + ColumnarToRow + InputAdapter + CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,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 + ReusedExchange [d_date_sk] #5 + HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] ColumnarToRow InputAdapter - CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_return_amount,cr_net_loss,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 - 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] #12 - WholeStageCodegen (9) - HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] + CometColumnarExchange #11 + RowToColumnar + WholeStageCodegen (7) + HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] + ColumnarToRow + InputAdapter + CometProject [cr_return_amount,cr_net_loss] + CometBroadcastHashJoin [cr_return_amount,cr_net_loss,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 + 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] ColumnarToRow InputAdapter - CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk] - CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_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 - ReusedExchange [d_date_sk] #5 - CometBroadcastExchange [wp_web_page_sk] #13 - CometFilter [wp_web_page_sk] - CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] - InputAdapter - 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] + CometColumnarExchange [wp_web_page_sk] #12 + RowToColumnar + WholeStageCodegen (9) + HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] + ColumnarToRow + InputAdapter + CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] + CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk] + CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_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 + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange [wp_web_page_sk] #13 + CometFilter [wp_web_page_sk] + CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter - Exchange [wp_web_page_sk] #15 - WholeStageCodegen (10) - HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] + 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] ColumnarToRow InputAdapter - CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] - CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] - CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk] - CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_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 - 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] #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 (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 #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 - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometColumnarExchange [wp_web_page_sk] #15 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] + ColumnarToRow + InputAdapter + CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] + CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk] + CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_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 + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel] #16 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange #17 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/explain.txt index c7ee5b1c9..8b2278f5f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/explain.txt @@ -1,74 +1,79 @@ == Physical Plan == -TakeOrderedAndProject (70) -+- * Project (69) - +- * SortMergeJoin Inner (68) - :- * Project (45) - : +- * SortMergeJoin Inner (44) - : :- * Sort (21) - : : +- * HashAggregate (20) - : : +- Exchange (19) - : : +- * HashAggregate (18) - : : +- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Project (14) - : : : +- * Filter (13) - : : : +- * SortMergeJoin LeftOuter (12) - : : : :- * Sort (5) - : : : : +- Exchange (4) - : : : : +- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- * Sort (11) - : : : +- Exchange (10) - : : : +- * ColumnarToRow (9) - : : : +- CometProject (8) - : : : +- CometFilter (7) - : : : +- CometScan parquet spark_catalog.default.store_returns (6) - : : +- ReusedExchange (15) - : +- * Sort (43) - : +- * Filter (42) - : +- * HashAggregate (41) - : +- Exchange (40) - : +- * HashAggregate (39) - : +- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * Project (35) - : : +- * Filter (34) - : : +- * SortMergeJoin LeftOuter (33) - : : :- * Sort (26) - : : : +- Exchange (25) - : : : +- * ColumnarToRow (24) - : : : +- CometFilter (23) - : : : +- CometScan parquet spark_catalog.default.web_sales (22) - : : +- * Sort (32) - : : +- Exchange (31) - : : +- * ColumnarToRow (30) - : : +- CometProject (29) - : : +- CometFilter (28) - : : +- CometScan parquet spark_catalog.default.web_returns (27) - : +- ReusedExchange (36) - +- * Sort (67) - +- * Filter (66) - +- * HashAggregate (65) - +- Exchange (64) - +- * HashAggregate (63) - +- * Project (62) - +- * BroadcastHashJoin Inner BuildRight (61) - :- * Project (59) - : +- * Filter (58) - : +- * SortMergeJoin LeftOuter (57) - : :- * Sort (50) - : : +- Exchange (49) - : : +- * ColumnarToRow (48) - : : +- CometFilter (47) - : : +- CometScan parquet spark_catalog.default.catalog_sales (46) - : +- * Sort (56) - : +- Exchange (55) - : +- * ColumnarToRow (54) - : +- CometProject (53) - : +- CometFilter (52) - : +- CometScan parquet spark_catalog.default.catalog_returns (51) - +- ReusedExchange (60) +TakeOrderedAndProject (75) ++- * Project (74) + +- * SortMergeJoin Inner (73) + :- * Project (49) + : +- * SortMergeJoin Inner (48) + : :- * Sort (24) + : : +- * HashAggregate (23) + : : +- * ColumnarToRow (22) + : : +- CometColumnarExchange (21) + : : +- RowToColumnar (20) + : : +- * HashAggregate (19) + : : +- * ColumnarToRow (18) + : : +- CometProject (17) + : : +- CometBroadcastHashJoin (16) + : : :- CometProject (12) + : : : +- CometFilter (11) + : : : +- CometSortMergeJoin (10) + : : : :- CometSort (4) + : : : : +- CometColumnarExchange (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometSort (9) + : : : +- CometColumnarExchange (8) + : : : +- CometProject (7) + : : : +- CometFilter (6) + : : : +- CometScan parquet spark_catalog.default.store_returns (5) + : : +- CometBroadcastExchange (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.date_dim (13) + : +- * Sort (47) + : +- * Filter (46) + : +- * HashAggregate (45) + : +- * ColumnarToRow (44) + : +- CometColumnarExchange (43) + : +- RowToColumnar (42) + : +- * HashAggregate (41) + : +- * ColumnarToRow (40) + : +- CometProject (39) + : +- CometBroadcastHashJoin (38) + : :- CometProject (36) + : : +- CometFilter (35) + : : +- CometSortMergeJoin (34) + : : :- CometSort (28) + : : : +- CometColumnarExchange (27) + : : : +- CometFilter (26) + : : : +- CometScan parquet spark_catalog.default.web_sales (25) + : : +- CometSort (33) + : : +- CometColumnarExchange (32) + : : +- CometProject (31) + : : +- CometFilter (30) + : : +- CometScan parquet spark_catalog.default.web_returns (29) + : +- ReusedExchange (37) + +- * Sort (72) + +- * Filter (71) + +- * HashAggregate (70) + +- * ColumnarToRow (69) + +- CometColumnarExchange (68) + +- RowToColumnar (67) + +- * HashAggregate (66) + +- * ColumnarToRow (65) + +- CometProject (64) + +- CometBroadcastHashJoin (63) + :- CometProject (61) + : +- CometFilter (60) + : +- CometSortMergeJoin (59) + : :- CometSort (53) + : : +- CometColumnarExchange (52) + : : +- CometFilter (51) + : : +- CometScan parquet spark_catalog.default.catalog_sales (50) + : +- CometSort (58) + : +- CometColumnarExchange (57) + : +- CometProject (56) + : +- CometFilter (55) + : +- CometScan parquet spark_catalog.default.catalog_returns (54) + +- ReusedExchange (62) (1) Scan parquet spark_catalog.default.store_sales @@ -83,93 +88,106 @@ ReadSchema: struct -(7) CometFilter +(6) CometFilter Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] Condition : (isnotnull(sr_ticket_number#10) AND isnotnull(sr_item_sk#9)) -(8) CometProject +(7) CometProject Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_item_sk#9, sr_ticket_number#10] -(9) ColumnarToRow [codegen id : 3] -Input [2]: [sr_item_sk#9, sr_ticket_number#10] - -(10) Exchange +(8) CometColumnarExchange Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(11) Sort [codegen id : 4] +(9) CometSort Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST], false, 0 +Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST] -(12) SortMergeJoin [codegen id : 6] -Left keys [2]: [ss_ticket_number#3, ss_item_sk#1] -Right keys [2]: [sr_ticket_number#10, sr_item_sk#9] -Join type: LeftOuter -Join condition: None +(10) CometSortMergeJoin +Left output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] +Right output [2]: [sr_item_sk#9, sr_ticket_number#10] +Arguments: [ss_ticket_number#3, ss_item_sk#1], [sr_ticket_number#10, sr_item_sk#9], LeftOuter -(13) Filter [codegen id : 6] +(11) CometFilter Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] Condition : isnull(sr_ticket_number#10) -(14) Project [codegen id : 6] -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] +(12) CometProject Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -(15) ReusedExchange [Reuses operator id: 74] +(13) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#12, d_year#13] +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 -(16) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None +(14) CometFilter +Input [2]: [d_date_sk#12, d_year#13] +Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) + +(15) CometBroadcastExchange +Input [2]: [d_date_sk#12, d_year#13] +Arguments: [d_date_sk#12, d_year#13] -(17) Project [codegen id : 6] -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] +(16) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] +Right output [2]: [d_date_sk#12, d_year#13] +Arguments: [ss_sold_date_sk#7], [d_date_sk#12], Inner, BuildRight + +(17) CometProject Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] + +(18) ColumnarToRow [codegen id : 1] +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] -(18) HashAggregate [codegen id : 6] +(19) HashAggregate [codegen id : 1] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesale_cost#5)), partial_sum(UnscaledValue(ss_sales_price#6))] Aggregate Attributes [3]: [sum#14, sum#15, sum#16] Results [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] -(19) Exchange +(20) RowToColumnar Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] -Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(20) HashAggregate [codegen id : 7] +(21) CometColumnarExchange +Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] +Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(22) ColumnarToRow [codegen id : 2] +Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] + +(23) HashAggregate [codegen id : 2] Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] Aggregate Attributes [3]: [sum(ss_quantity#4)#20, sum(UnscaledValue(ss_wholesale_cost#5))#21, sum(UnscaledValue(ss_sales_price#6))#22] Results [6]: [d_year#13 AS ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, sum(ss_quantity#4)#20 AS ss_qty#24, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#5))#21,17,2) AS ss_wc#25, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#22,17,2) AS ss_sp#26] -(21) Sort [codegen id : 7] +(24) Sort [codegen id : 2] Input [6]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] Arguments: [ss_sold_year#23 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], false, 0 -(22) Scan parquet spark_catalog.default.web_sales +(25) Scan parquet spark_catalog.default.web_sales Output [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] @@ -177,111 +195,112 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(23) CometFilter +(26) CometFilter Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] Condition : (isnotnull(ws_item_sk#27) AND isnotnull(ws_bill_customer_sk#28)) -(24) ColumnarToRow [codegen id : 8] -Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] - -(25) Exchange +(27) CometColumnarExchange Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Arguments: hashpartitioning(ws_order_number#29, ws_item_sk#27, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: hashpartitioning(ws_order_number#29, ws_item_sk#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(26) Sort [codegen id : 9] +(28) CometSort Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Arguments: [ws_order_number#29 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST], false, 0 +Arguments: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33], [ws_order_number#29 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST] -(27) Scan parquet spark_catalog.default.web_returns +(29) Scan parquet spark_catalog.default.web_returns Output [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct -(28) CometFilter +(30) CometFilter Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] Condition : (isnotnull(wr_order_number#36) AND isnotnull(wr_item_sk#35)) -(29) CometProject +(31) CometProject Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] Arguments: [wr_item_sk#35, wr_order_number#36], [wr_item_sk#35, wr_order_number#36] -(30) ColumnarToRow [codegen id : 10] +(32) CometColumnarExchange Input [2]: [wr_item_sk#35, wr_order_number#36] +Arguments: hashpartitioning(wr_order_number#36, wr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(31) Exchange +(33) CometSort Input [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: hashpartitioning(wr_order_number#36, wr_item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Arguments: [wr_item_sk#35, wr_order_number#36], [wr_order_number#36 ASC NULLS FIRST, wr_item_sk#35 ASC NULLS FIRST] -(32) Sort [codegen id : 11] -Input [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: [wr_order_number#36 ASC NULLS FIRST, wr_item_sk#35 ASC NULLS FIRST], false, 0 - -(33) SortMergeJoin [codegen id : 13] -Left keys [2]: [ws_order_number#29, ws_item_sk#27] -Right keys [2]: [wr_order_number#36, wr_item_sk#35] -Join type: LeftOuter -Join condition: None +(34) CometSortMergeJoin +Left output [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Right output [2]: [wr_item_sk#35, wr_order_number#36] +Arguments: [ws_order_number#29, ws_item_sk#27], [wr_order_number#36, wr_item_sk#35], LeftOuter -(34) Filter [codegen id : 13] +(35) CometFilter Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] Condition : isnull(wr_order_number#36) -(35) Project [codegen id : 13] -Output [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +(36) CometProject Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] +Arguments: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33], [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -(36) ReusedExchange [Reuses operator id: 74] +(37) ReusedExchange [Reuses operator id: 15] Output [2]: [d_date_sk#38, d_year#39] -(37) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#38] -Join type: Inner -Join condition: None +(38) CometBroadcastHashJoin +Left output [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Right output [2]: [d_date_sk#38, d_year#39] +Arguments: [ws_sold_date_sk#33], [d_date_sk#38], Inner, BuildRight -(38) Project [codegen id : 13] -Output [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] +(39) CometProject Input [8]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, d_date_sk#38, d_year#39] +Arguments: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39], [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] + +(40) ColumnarToRow [codegen id : 3] +Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] -(39) HashAggregate [codegen id : 13] +(41) HashAggregate [codegen id : 3] Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] Functions [3]: [partial_sum(ws_quantity#30), partial_sum(UnscaledValue(ws_wholesale_cost#31)), partial_sum(UnscaledValue(ws_sales_price#32))] Aggregate Attributes [3]: [sum#40, sum#41, sum#42] Results [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] -(40) Exchange +(42) RowToColumnar +Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] + +(43) CometColumnarExchange Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] -Arguments: hashpartitioning(d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Arguments: hashpartitioning(d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(41) HashAggregate [codegen id : 14] +(44) ColumnarToRow [codegen id : 4] +Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] + +(45) HashAggregate [codegen id : 4] Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] Functions [3]: [sum(ws_quantity#30), sum(UnscaledValue(ws_wholesale_cost#31)), sum(UnscaledValue(ws_sales_price#32))] Aggregate Attributes [3]: [sum(ws_quantity#30)#46, sum(UnscaledValue(ws_wholesale_cost#31))#47, sum(UnscaledValue(ws_sales_price#32))#48] Results [6]: [d_year#39 AS ws_sold_year#49, ws_item_sk#27, ws_bill_customer_sk#28 AS ws_customer_sk#50, sum(ws_quantity#30)#46 AS ws_qty#51, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#31))#47,17,2) AS ws_wc#52, MakeDecimal(sum(UnscaledValue(ws_sales_price#32))#48,17,2) AS ws_sp#53] -(42) Filter [codegen id : 14] +(46) Filter [codegen id : 4] Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] Condition : (coalesce(ws_qty#51, 0) > 0) -(43) Sort [codegen id : 14] +(47) Sort [codegen id : 4] Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] Arguments: [ws_sold_year#49 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST, ws_customer_sk#50 ASC NULLS FIRST], false, 0 -(44) SortMergeJoin [codegen id : 15] +(48) SortMergeJoin [codegen id : 5] Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] Right keys [3]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50] Join type: Inner Join condition: None -(45) Project [codegen id : 15] +(49) Project [codegen id : 5] Output [9]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53] Input [12]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] -(46) Scan parquet spark_catalog.default.catalog_sales +(50) Scan parquet spark_catalog.default.catalog_sales Output [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] Batched: true Location: InMemoryFileIndex [] @@ -289,143 +308,144 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(47) CometFilter +(51) CometFilter Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] Condition : (isnotnull(cs_item_sk#55) AND isnotnull(cs_bill_customer_sk#54)) -(48) ColumnarToRow [codegen id : 16] -Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] - -(49) Exchange +(52) CometColumnarExchange Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Arguments: hashpartitioning(cs_order_number#56, cs_item_sk#55, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Arguments: hashpartitioning(cs_order_number#56, cs_item_sk#55, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(50) Sort [codegen id : 17] +(53) CometSort Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Arguments: [cs_order_number#56 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST], false, 0 +Arguments: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60], [cs_order_number#56 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST] -(51) Scan parquet spark_catalog.default.catalog_returns +(54) Scan parquet spark_catalog.default.catalog_returns Output [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(52) CometFilter +(55) CometFilter Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] Condition : (isnotnull(cr_order_number#63) AND isnotnull(cr_item_sk#62)) -(53) CometProject +(56) CometProject Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] Arguments: [cr_item_sk#62, cr_order_number#63], [cr_item_sk#62, cr_order_number#63] -(54) ColumnarToRow [codegen id : 18] -Input [2]: [cr_item_sk#62, cr_order_number#63] - -(55) Exchange +(57) CometColumnarExchange Input [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: hashpartitioning(cr_order_number#63, cr_item_sk#62, 5), ENSURE_REQUIREMENTS, [plan_id=8] +Arguments: hashpartitioning(cr_order_number#63, cr_item_sk#62, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(56) Sort [codegen id : 19] +(58) CometSort Input [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: [cr_order_number#63 ASC NULLS FIRST, cr_item_sk#62 ASC NULLS FIRST], false, 0 +Arguments: [cr_item_sk#62, cr_order_number#63], [cr_order_number#63 ASC NULLS FIRST, cr_item_sk#62 ASC NULLS FIRST] -(57) SortMergeJoin [codegen id : 21] -Left keys [2]: [cs_order_number#56, cs_item_sk#55] -Right keys [2]: [cr_order_number#63, cr_item_sk#62] -Join type: LeftOuter -Join condition: None +(59) CometSortMergeJoin +Left output [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +Right output [2]: [cr_item_sk#62, cr_order_number#63] +Arguments: [cs_order_number#56, cs_item_sk#55], [cr_order_number#63, cr_item_sk#62], LeftOuter -(58) Filter [codegen id : 21] +(60) CometFilter Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] Condition : isnull(cr_order_number#63) -(59) Project [codegen id : 21] -Output [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +(61) CometProject Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] +Arguments: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60], [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -(60) ReusedExchange [Reuses operator id: 74] +(62) ReusedExchange [Reuses operator id: 15] Output [2]: [d_date_sk#65, d_year#66] -(61) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [cs_sold_date_sk#60] -Right keys [1]: [d_date_sk#65] -Join type: Inner -Join condition: None +(63) CometBroadcastHashJoin +Left output [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +Right output [2]: [d_date_sk#65, d_year#66] +Arguments: [cs_sold_date_sk#60], [d_date_sk#65], Inner, BuildRight -(62) Project [codegen id : 21] -Output [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] +(64) CometProject Input [8]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, d_date_sk#65, d_year#66] +Arguments: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66], [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] -(63) HashAggregate [codegen id : 21] +(65) ColumnarToRow [codegen id : 6] +Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] + +(66) HashAggregate [codegen id : 6] Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] Functions [3]: [partial_sum(cs_quantity#57), partial_sum(UnscaledValue(cs_wholesale_cost#58)), partial_sum(UnscaledValue(cs_sales_price#59))] Aggregate Attributes [3]: [sum#67, sum#68, sum#69] Results [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] -(64) Exchange +(67) RowToColumnar +Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] + +(68) CometColumnarExchange +Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] +Arguments: hashpartitioning(d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(69) ColumnarToRow [codegen id : 7] Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] -Arguments: hashpartitioning(d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(65) HashAggregate [codegen id : 22] +(70) HashAggregate [codegen id : 7] Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] Functions [3]: [sum(cs_quantity#57), sum(UnscaledValue(cs_wholesale_cost#58)), sum(UnscaledValue(cs_sales_price#59))] Aggregate Attributes [3]: [sum(cs_quantity#57)#73, sum(UnscaledValue(cs_wholesale_cost#58))#74, sum(UnscaledValue(cs_sales_price#59))#75] Results [6]: [d_year#66 AS cs_sold_year#76, cs_item_sk#55, cs_bill_customer_sk#54 AS cs_customer_sk#77, sum(cs_quantity#57)#73 AS cs_qty#78, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#58))#74,17,2) AS cs_wc#79, MakeDecimal(sum(UnscaledValue(cs_sales_price#59))#75,17,2) AS cs_sp#80] -(66) Filter [codegen id : 22] +(71) Filter [codegen id : 7] Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] Condition : (coalesce(cs_qty#78, 0) > 0) -(67) Sort [codegen id : 22] +(72) Sort [codegen id : 7] Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] Arguments: [cs_sold_year#76 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST, cs_customer_sk#77 ASC NULLS FIRST], false, 0 -(68) SortMergeJoin [codegen id : 23] +(73) SortMergeJoin [codegen id : 8] Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] Right keys [3]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77] Join type: Inner Join condition: None -(69) Project [codegen id : 23] +(74) Project [codegen id : 8] Output [13]: [round((cast(ss_qty#24 as double) / cast(coalesce((ws_qty#51 + cs_qty#78), 1) as double)), 2) AS ratio#81, ss_qty#24 AS store_qty#82, ss_wc#25 AS store_wholesale_cost#83, ss_sp#26 AS store_sales_price#84, (coalesce(ws_qty#51, 0) + coalesce(cs_qty#78, 0)) AS other_chan_qty#85, (coalesce(ws_wc#52, 0.00) + coalesce(cs_wc#79, 0.00)) AS other_chan_wholesale_cost#86, (coalesce(ws_sp#53, 0.00) + coalesce(cs_sp#80, 0.00)) AS other_chan_sales_price#87, ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] Input [15]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53, cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] -(70) TakeOrderedAndProject +(75) TakeOrderedAndProject Input [13]: [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87, ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] Arguments: 100, [ss_sold_year#23 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#24 DESC NULLS LAST, ss_wc#25 DESC NULLS LAST, ss_sp#26 DESC NULLS LAST, other_chan_qty#85 ASC NULLS FIRST, other_chan_wholesale_cost#86 ASC NULLS FIRST, other_chan_sales_price#87 ASC NULLS FIRST, ratio#81 ASC NULLS FIRST], [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (74) -+- * ColumnarToRow (73) - +- CometFilter (72) - +- CometScan parquet spark_catalog.default.date_dim (71) +BroadcastExchange (79) ++- * ColumnarToRow (78) + +- CometFilter (77) + +- CometScan parquet spark_catalog.default.date_dim (76) -(71) Scan parquet spark_catalog.default.date_dim +(76) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#12, d_year#13] 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 -(72) CometFilter +(77) CometFilter Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(73) ColumnarToRow [codegen id : 1] +(78) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] -(74) BroadcastExchange +(79) BroadcastExchange Input [2]: [d_date_sk#12, d_year#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 25 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 50 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/simplified.txt index b8e6e0605..e903d5b7f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/simplified.txt @@ -1,127 +1,102 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ratio,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (23) + WholeStageCodegen (8) Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp,ss_sold_year,ss_item_sk,ss_customer_sk] SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,cs_sold_year,cs_item_sk,cs_customer_sk] InputAdapter - WholeStageCodegen (15) + WholeStageCodegen (5) Project [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ws_sold_year,ws_item_sk,ws_customer_sk] InputAdapter - WholeStageCodegen (7) + WholeStageCodegen (2) Sort [ss_sold_year,ss_item_sk,ss_customer_sk] HashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] [sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price)),ss_sold_year,ss_qty,ss_wc,ss_sp,sum,sum,sum] - InputAdapter - Exchange [d_year,ss_item_sk,ss_customer_sk] #1 - WholeStageCodegen (6) - HashAggregate [d_year,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] [sum,sum,sum,sum,sum,sum] - Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - Filter [sr_ticket_number] - SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - WholeStageCodegen (2) - Sort [ss_ticket_number,ss_item_sk] - InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - WholeStageCodegen (4) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - Exchange [sr_ticket_number,sr_item_sk] #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + ColumnarToRow + InputAdapter + CometColumnarExchange [d_year,ss_item_sk,ss_customer_sk] #1 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [d_year,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] [sum,sum,sum,sum,sum,sum] + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #2 + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometSort [sr_item_sk,sr_ticket_number] + CometColumnarExchange [sr_ticket_number,sr_item_sk] #4 + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #5 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - WholeStageCodegen (14) + WholeStageCodegen (4) Sort [ws_sold_year,ws_item_sk,ws_customer_sk] Filter [ws_qty] HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] [sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price)),ws_sold_year,ws_customer_sk,ws_qty,ws_wc,ws_sp,sum,sum,sum] - InputAdapter - Exchange [d_year,ws_item_sk,ws_bill_customer_sk] #5 - WholeStageCodegen (13) - HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] [sum,sum,sum,sum,sum,sum] - Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - Filter [wr_order_number] - SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - InputAdapter - WholeStageCodegen (9) - Sort [ws_order_number,ws_item_sk] - InputAdapter - Exchange [ws_order_number,ws_item_sk] #6 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - WholeStageCodegen (11) - Sort [wr_order_number,wr_item_sk] - InputAdapter - Exchange [wr_order_number,wr_item_sk] #7 - WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + ColumnarToRow + InputAdapter + CometColumnarExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] [sum,sum,sum,sum,sum,sum] + ColumnarToRow + InputAdapter + CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] + CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,d_date_sk,d_year] + CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] + CometSortMergeJoin [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] + CometSort [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometColumnarExchange [ws_order_number,ws_item_sk] #7 + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_item_sk,wr_order_number] + CometColumnarExchange [wr_order_number,wr_item_sk] #8 + CometProject [wr_item_sk,wr_order_number] + CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] + ReusedExchange [d_date_sk,d_year] #5 InputAdapter - WholeStageCodegen (22) + WholeStageCodegen (7) Sort [cs_sold_year,cs_item_sk,cs_customer_sk] Filter [cs_qty] HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] [sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price)),cs_sold_year,cs_customer_sk,cs_qty,cs_wc,cs_sp,sum,sum,sum] - InputAdapter - Exchange [d_year,cs_item_sk,cs_bill_customer_sk] #8 - WholeStageCodegen (21) - HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] [sum,sum,sum,sum,sum,sum] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - Filter [cr_order_number] - SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - InputAdapter - WholeStageCodegen (17) - Sort [cs_order_number,cs_item_sk] - InputAdapter - Exchange [cs_order_number,cs_item_sk] #9 - WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - WholeStageCodegen (19) - Sort [cr_order_number,cr_item_sk] - InputAdapter - Exchange [cr_order_number,cr_item_sk] #10 - WholeStageCodegen (18) - ColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + ColumnarToRow + InputAdapter + CometColumnarExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] [sum,sum,sum,sum,sum,sum] + ColumnarToRow + InputAdapter + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] + CometSortMergeJoin [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] + CometSort [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + CometColumnarExchange [cs_order_number,cs_item_sk] #10 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [cr_item_sk,cr_order_number] + CometColumnarExchange [cr_order_number,cr_item_sk] #11 + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] + ReusedExchange [d_date_sk,d_year] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/explain.txt index 8f47f4f0e..f52b07ad6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/explain.txt @@ -1,124 +1,135 @@ == Physical Plan == -TakeOrderedAndProject (120) -+- * HashAggregate (119) - +- Exchange (118) - +- * HashAggregate (117) - +- Union (116) - :- * HashAggregate (105) - : +- Exchange (104) - : +- * HashAggregate (103) - : +- Union (102) - : :- * HashAggregate (39) - : : +- Exchange (38) - : : +- * HashAggregate (37) - : : +- * Project (36) - : : +- * BroadcastHashJoin Inner BuildRight (35) - : : :- * Project (29) - : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : :- * Project (22) - : : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : : :- * Project (16) - : : : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : : : :- * Project (13) - : : : : : : +- * SortMergeJoin LeftOuter (12) - : : : : : : :- * Sort (5) - : : : : : : : +- Exchange (4) - : : : : : : : +- * ColumnarToRow (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : +- * Sort (11) - : : : : : : +- Exchange (10) - : : : : : : +- * ColumnarToRow (9) - : : : : : : +- CometProject (8) - : : : : : : +- CometFilter (7) - : : : : : : +- CometScan parquet spark_catalog.default.store_returns (6) - : : : : : +- ReusedExchange (14) - : : : : +- BroadcastExchange (20) - : : : : +- * ColumnarToRow (19) - : : : : +- CometFilter (18) - : : : : +- CometScan parquet spark_catalog.default.store (17) - : : : +- BroadcastExchange (27) - : : : +- * ColumnarToRow (26) - : : : +- CometProject (25) - : : : +- CometFilter (24) - : : : +- CometScan parquet spark_catalog.default.item (23) - : : +- BroadcastExchange (34) - : : +- * ColumnarToRow (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan parquet spark_catalog.default.promotion (30) - : :- * HashAggregate (70) - : : +- Exchange (69) - : : +- * HashAggregate (68) - : : +- * Project (67) - : : +- * BroadcastHashJoin Inner BuildRight (66) - : : :- * Project (64) - : : : +- * BroadcastHashJoin Inner BuildRight (63) - : : : :- * Project (61) - : : : : +- * BroadcastHashJoin Inner BuildRight (60) - : : : : :- * Project (55) - : : : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : : : :- * Project (52) - : : : : : : +- * SortMergeJoin LeftOuter (51) - : : : : : : :- * Sort (44) - : : : : : : : +- Exchange (43) - : : : : : : : +- * ColumnarToRow (42) - : : : : : : : +- CometFilter (41) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (40) - : : : : : : +- * Sort (50) - : : : : : : +- Exchange (49) - : : : : : : +- * ColumnarToRow (48) - : : : : : : +- CometProject (47) - : : : : : : +- CometFilter (46) - : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (45) - : : : : : +- ReusedExchange (53) - : : : : +- BroadcastExchange (59) - : : : : +- * ColumnarToRow (58) - : : : : +- CometFilter (57) - : : : : +- CometScan parquet spark_catalog.default.catalog_page (56) - : : : +- ReusedExchange (62) - : : +- ReusedExchange (65) - : +- * HashAggregate (101) - : +- Exchange (100) - : +- * HashAggregate (99) - : +- * Project (98) - : +- * BroadcastHashJoin Inner BuildRight (97) - : :- * Project (95) - : : +- * BroadcastHashJoin Inner BuildRight (94) - : : :- * Project (92) - : : : +- * BroadcastHashJoin Inner BuildRight (91) - : : : :- * Project (86) - : : : : +- * BroadcastHashJoin Inner BuildRight (85) - : : : : :- * Project (83) - : : : : : +- * SortMergeJoin LeftOuter (82) - : : : : : :- * Sort (75) - : : : : : : +- Exchange (74) - : : : : : : +- * ColumnarToRow (73) - : : : : : : +- CometFilter (72) - : : : : : : +- CometScan parquet spark_catalog.default.web_sales (71) - : : : : : +- * Sort (81) - : : : : : +- Exchange (80) - : : : : : +- * ColumnarToRow (79) - : : : : : +- CometProject (78) - : : : : : +- CometFilter (77) - : : : : : +- CometScan parquet spark_catalog.default.web_returns (76) - : : : : +- ReusedExchange (84) - : : : +- BroadcastExchange (90) - : : : +- * ColumnarToRow (89) - : : : +- CometFilter (88) - : : : +- CometScan parquet spark_catalog.default.web_site (87) - : : +- ReusedExchange (93) - : +- ReusedExchange (96) - :- * HashAggregate (110) - : +- Exchange (109) - : +- * HashAggregate (108) - : +- * HashAggregate (107) - : +- ReusedExchange (106) - +- * HashAggregate (115) - +- Exchange (114) - +- * HashAggregate (113) - +- * HashAggregate (112) - +- ReusedExchange (111) +* ColumnarToRow (131) ++- CometTakeOrderedAndProject (130) + +- CometHashAggregate (129) + +- CometColumnarExchange (128) + +- RowToColumnar (127) + +- * HashAggregate (126) + +- Union (125) + :- * HashAggregate (108) + : +- * ColumnarToRow (107) + : +- CometColumnarExchange (106) + : +- RowToColumnar (105) + : +- * HashAggregate (104) + : +- Union (103) + : :- * HashAggregate (40) + : : +- * ColumnarToRow (39) + : : +- CometColumnarExchange (38) + : : +- RowToColumnar (37) + : : +- * HashAggregate (36) + : : +- * ColumnarToRow (35) + : : +- CometProject (34) + : : +- CometBroadcastHashJoin (33) + : : :- CometProject (28) + : : : +- CometBroadcastHashJoin (27) + : : : :- CometProject (22) + : : : : +- CometBroadcastHashJoin (21) + : : : : :- CometProject (17) + : : : : : +- CometBroadcastHashJoin (16) + : : : : : :- CometProject (11) + : : : : : : +- CometSortMergeJoin (10) + : : : : : : :- CometSort (4) + : : : : : : : +- CometColumnarExchange (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- CometSort (9) + : : : : : : +- CometColumnarExchange (8) + : : : : : : +- CometProject (7) + : : : : : : +- CometFilter (6) + : : : : : : +- CometScan parquet spark_catalog.default.store_returns (5) + : : : : : +- CometBroadcastExchange (15) + : : : : : +- CometProject (14) + : : : : : +- CometFilter (13) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (12) + : : : : +- CometBroadcastExchange (20) + : : : : +- CometFilter (19) + : : : : +- CometScan parquet spark_catalog.default.store (18) + : : : +- CometBroadcastExchange (26) + : : : +- CometProject (25) + : : : +- CometFilter (24) + : : : +- CometScan parquet spark_catalog.default.item (23) + : : +- CometBroadcastExchange (32) + : : +- CometProject (31) + : : +- CometFilter (30) + : : +- CometScan parquet spark_catalog.default.promotion (29) + : :- * HashAggregate (71) + : : +- * ColumnarToRow (70) + : : +- CometColumnarExchange (69) + : : +- RowToColumnar (68) + : : +- * HashAggregate (67) + : : +- * ColumnarToRow (66) + : : +- CometProject (65) + : : +- CometBroadcastHashJoin (64) + : : :- CometProject (62) + : : : +- CometBroadcastHashJoin (61) + : : : :- CometProject (59) + : : : : +- CometBroadcastHashJoin (58) + : : : : :- CometProject (54) + : : : : : +- CometBroadcastHashJoin (53) + : : : : : :- CometProject (51) + : : : : : : +- CometSortMergeJoin (50) + : : : : : : :- CometSort (44) + : : : : : : : +- CometColumnarExchange (43) + : : : : : : : +- CometFilter (42) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (41) + : : : : : : +- CometSort (49) + : : : : : : +- CometColumnarExchange (48) + : : : : : : +- CometProject (47) + : : : : : : +- CometFilter (46) + : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (45) + : : : : : +- ReusedExchange (52) + : : : : +- CometBroadcastExchange (57) + : : : : +- CometFilter (56) + : : : : +- CometScan parquet spark_catalog.default.catalog_page (55) + : : : +- ReusedExchange (60) + : : +- ReusedExchange (63) + : +- * HashAggregate (102) + : +- * ColumnarToRow (101) + : +- CometColumnarExchange (100) + : +- RowToColumnar (99) + : +- * HashAggregate (98) + : +- * ColumnarToRow (97) + : +- CometProject (96) + : +- CometBroadcastHashJoin (95) + : :- CometProject (93) + : : +- CometBroadcastHashJoin (92) + : : :- CometProject (90) + : : : +- CometBroadcastHashJoin (89) + : : : :- CometProject (85) + : : : : +- CometBroadcastHashJoin (84) + : : : : :- CometProject (82) + : : : : : +- CometSortMergeJoin (81) + : : : : : :- CometSort (75) + : : : : : : +- CometColumnarExchange (74) + : : : : : : +- CometFilter (73) + : : : : : : +- CometScan parquet spark_catalog.default.web_sales (72) + : : : : : +- CometSort (80) + : : : : : +- CometColumnarExchange (79) + : : : : : +- CometProject (78) + : : : : : +- CometFilter (77) + : : : : : +- CometScan parquet spark_catalog.default.web_returns (76) + : : : : +- ReusedExchange (83) + : : : +- CometBroadcastExchange (88) + : : : +- CometFilter (87) + : : : +- CometScan parquet spark_catalog.default.web_site (86) + : : +- ReusedExchange (91) + : +- ReusedExchange (94) + :- * HashAggregate (116) + : +- * ColumnarToRow (115) + : +- CometColumnarExchange (114) + : +- RowToColumnar (113) + : +- * HashAggregate (112) + : +- * HashAggregate (111) + : +- * ColumnarToRow (110) + : +- ReusedExchange (109) + +- * HashAggregate (124) + +- * ColumnarToRow (123) + +- CometColumnarExchange (122) + +- RowToColumnar (121) + +- * HashAggregate (120) + +- * HashAggregate (119) + +- * ColumnarToRow (118) + +- ReusedExchange (117) (1) Scan parquet spark_catalog.default.store_sales @@ -133,598 +144,621 @@ ReadSchema: struct -(7) CometFilter +(6) CometFilter Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] Condition : (isnotnull(sr_item_sk#9) AND isnotnull(sr_ticket_number#10)) -(8) CometProject +(7) CometProject Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -(9) ColumnarToRow [codegen id : 3] -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] - -(10) Exchange +(8) CometColumnarExchange Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(11) Sort [codegen id : 4] +(9) CometSort Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST], false, 0 +Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST] -(12) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_item_sk#1, ss_ticket_number#4] -Right keys [2]: [sr_item_sk#9, sr_ticket_number#10] -Join type: LeftOuter -Join condition: None +(10) CometSortMergeJoin +Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] +Right output [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] +Arguments: [ss_item_sk#1, ss_ticket_number#4], [sr_item_sk#9, sr_ticket_number#10], LeftOuter -(13) Project [codegen id : 9] -Output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] +(11) CometProject Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] +Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] -(14) ReusedExchange [Reuses operator id: 125] -Output [1]: [d_date_sk#14] +(12) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#15] +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 -(15) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None +(13) CometFilter +Input [2]: [d_date_sk#14, d_date#15] +Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) -(16) Project [codegen id : 9] -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] +(14) CometProject +Input [2]: [d_date_sk#14, d_date#15] +Arguments: [d_date_sk#14], [d_date_sk#14] + +(15) CometBroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: [d_date_sk#14] + +(16) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] +Right output [1]: [d_date_sk#14] +Arguments: [ss_sold_date_sk#7], [d_date_sk#14], Inner, BuildRight + +(17) CometProject Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#14] +Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] -(17) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#15, s_store_id#16] +(18) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#16, s_store_id#17] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(18) CometFilter -Input [2]: [s_store_sk#15, s_store_id#16] -Condition : isnotnull(s_store_sk#15) - -(19) ColumnarToRow [codegen id : 6] -Input [2]: [s_store_sk#15, s_store_id#16] +(19) CometFilter +Input [2]: [s_store_sk#16, s_store_id#17] +Condition : isnotnull(s_store_sk#16) -(20) BroadcastExchange -Input [2]: [s_store_sk#15, s_store_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(20) CometBroadcastExchange +Input [2]: [s_store_sk#16, s_store_id#17] +Arguments: [s_store_sk#16, s_store_id#17] -(21) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#15] -Join type: Inner -Join condition: None +(21) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] +Right output [2]: [s_store_sk#16, s_store_id#17] +Arguments: [ss_store_sk#2], [s_store_sk#16], Inner, BuildRight -(22) Project [codegen id : 9] -Output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#15, s_store_id#16] +(22) CometProject +Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#16, s_store_id#17] +Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17], [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] (23) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#17, i_current_price#18] +Output [2]: [i_item_sk#18, i_current_price#19] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), GreaterThan(i_current_price,50.00), IsNotNull(i_item_sk)] ReadSchema: struct (24) CometFilter -Input [2]: [i_item_sk#17, i_current_price#18] -Condition : ((isnotnull(i_current_price#18) AND (i_current_price#18 > 50.00)) AND isnotnull(i_item_sk#17)) +Input [2]: [i_item_sk#18, i_current_price#19] +Condition : ((isnotnull(i_current_price#19) AND (i_current_price#19 > 50.00)) AND isnotnull(i_item_sk#18)) (25) CometProject -Input [2]: [i_item_sk#17, i_current_price#18] -Arguments: [i_item_sk#17], [i_item_sk#17] +Input [2]: [i_item_sk#18, i_current_price#19] +Arguments: [i_item_sk#18], [i_item_sk#18] -(26) ColumnarToRow [codegen id : 7] -Input [1]: [i_item_sk#17] +(26) CometBroadcastExchange +Input [1]: [i_item_sk#18] +Arguments: [i_item_sk#18] -(27) BroadcastExchange -Input [1]: [i_item_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(27) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] +Right output [1]: [i_item_sk#18] +Arguments: [ss_item_sk#1], [i_item_sk#18], Inner, BuildRight -(28) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#17] -Join type: Inner -Join condition: None +(28) CometProject +Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17, i_item_sk#18] +Arguments: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17], [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] -(29) Project [codegen id : 9] -Output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16, i_item_sk#17] - -(30) Scan parquet spark_catalog.default.promotion -Output [2]: [p_promo_sk#19, p_channel_tv#20] +(29) Scan parquet spark_catalog.default.promotion +Output [2]: [p_promo_sk#20, p_channel_tv#21] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_channel_tv), EqualTo(p_channel_tv,N), IsNotNull(p_promo_sk)] ReadSchema: struct -(31) CometFilter -Input [2]: [p_promo_sk#19, p_channel_tv#20] -Condition : ((isnotnull(p_channel_tv#20) AND (p_channel_tv#20 = N)) AND isnotnull(p_promo_sk#19)) +(30) CometFilter +Input [2]: [p_promo_sk#20, p_channel_tv#21] +Condition : ((isnotnull(p_channel_tv#21) AND (p_channel_tv#21 = N)) AND isnotnull(p_promo_sk#20)) -(32) CometProject -Input [2]: [p_promo_sk#19, p_channel_tv#20] -Arguments: [p_promo_sk#19], [p_promo_sk#19] +(31) CometProject +Input [2]: [p_promo_sk#20, p_channel_tv#21] +Arguments: [p_promo_sk#20], [p_promo_sk#20] -(33) ColumnarToRow [codegen id : 8] -Input [1]: [p_promo_sk#19] +(32) CometBroadcastExchange +Input [1]: [p_promo_sk#20] +Arguments: [p_promo_sk#20] -(34) BroadcastExchange -Input [1]: [p_promo_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(33) CometBroadcastHashJoin +Left output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] +Right output [1]: [p_promo_sk#20] +Arguments: [ss_promo_sk#3], [p_promo_sk#20], Inner, BuildRight -(35) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_promo_sk#3] -Right keys [1]: [p_promo_sk#19] -Join type: Inner -Join condition: None +(34) CometProject +Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17, p_promo_sk#20] +Arguments: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17], [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] -(36) Project [codegen id : 9] -Output [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] -Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16, p_promo_sk#19] +(35) ColumnarToRow [codegen id : 1] +Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] -(37) HashAggregate [codegen id : 9] -Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] -Keys [1]: [s_store_id#16] +(36) HashAggregate [codegen id : 1] +Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] +Keys [1]: [s_store_id#17] Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] -Aggregate Attributes [5]: [sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -Results [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] +Aggregate Attributes [5]: [sum#22, sum#23, isEmpty#24, sum#25, isEmpty#26] +Results [6]: [s_store_id#17, sum#27, sum#28, isEmpty#29, sum#30, isEmpty#31] + +(37) RowToColumnar +Input [6]: [s_store_id#17, sum#27, sum#28, isEmpty#29, sum#30, isEmpty#31] -(38) Exchange -Input [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] -Arguments: hashpartitioning(s_store_id#16, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(38) CometColumnarExchange +Input [6]: [s_store_id#17, sum#27, sum#28, isEmpty#29, sum#30, isEmpty#31] +Arguments: hashpartitioning(s_store_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(39) HashAggregate [codegen id : 10] -Input [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] -Keys [1]: [s_store_id#16] +(39) ColumnarToRow [codegen id : 2] +Input [6]: [s_store_id#17, sum#27, sum#28, isEmpty#29, sum#30, isEmpty#31] + +(40) HashAggregate [codegen id : 2] +Input [6]: [s_store_id#17, sum#27, sum#28, isEmpty#29, sum#30, isEmpty#31] +Keys [1]: [s_store_id#17] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#5))#31, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#32, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#33] -Results [5]: [store channel AS channel#34, concat(store, s_store_id#16) AS id#35, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#31,17,2) AS sales#36, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#32 AS returns#37, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#33 AS profit#38] +Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#5))#32, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#33, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#34] +Results [5]: [store channel AS channel#35, concat(store, s_store_id#17) AS id#36, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#32,17,2) AS sales#37, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#33 AS returns#38, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#34 AS profit#39] -(40) Scan parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +(41) Scan parquet spark_catalog.default.catalog_sales +Output [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#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), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] ReadSchema: struct -(41) CometFilter -Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Condition : ((isnotnull(cs_catalog_page_sk#39) AND isnotnull(cs_item_sk#40)) AND isnotnull(cs_promo_sk#41)) - -(42) ColumnarToRow [codegen id : 11] -Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +(42) CometFilter +Input [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] +Condition : ((isnotnull(cs_catalog_page_sk#40) AND isnotnull(cs_item_sk#41)) AND isnotnull(cs_promo_sk#42)) -(43) Exchange -Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Arguments: hashpartitioning(cs_item_sk#40, cs_order_number#42, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(43) CometColumnarExchange +Input [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] +Arguments: hashpartitioning(cs_item_sk#41, cs_order_number#43, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(44) Sort [codegen id : 12] -Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Arguments: [cs_item_sk#40 ASC NULLS FIRST, cs_order_number#42 ASC NULLS FIRST], false, 0 +(44) CometSort +Input [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] +Arguments: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46], [cs_item_sk#41 ASC NULLS FIRST, cs_order_number#43 ASC NULLS FIRST] (45) Scan parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] +Output [5]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51, cr_returned_date_sk#52] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct (46) CometFilter -Input [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] -Condition : (isnotnull(cr_item_sk#47) AND isnotnull(cr_order_number#48)) +Input [5]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51, cr_returned_date_sk#52] +Condition : (isnotnull(cr_item_sk#48) AND isnotnull(cr_order_number#49)) (47) CometProject -Input [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] -Arguments: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50], [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] - -(48) ColumnarToRow [codegen id : 13] -Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] +Input [5]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51, cr_returned_date_sk#52] +Arguments: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51], [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51] -(49) Exchange -Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -Arguments: hashpartitioning(cr_item_sk#47, cr_order_number#48, 5), ENSURE_REQUIREMENTS, [plan_id=8] +(48) CometColumnarExchange +Input [4]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51] +Arguments: hashpartitioning(cr_item_sk#48, cr_order_number#49, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(50) Sort [codegen id : 14] -Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -Arguments: [cr_item_sk#47 ASC NULLS FIRST, cr_order_number#48 ASC NULLS FIRST], false, 0 +(49) CometSort +Input [4]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51] +Arguments: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51], [cr_item_sk#48 ASC NULLS FIRST, cr_order_number#49 ASC NULLS FIRST] -(51) SortMergeJoin [codegen id : 19] -Left keys [2]: [cs_item_sk#40, cs_order_number#42] -Right keys [2]: [cr_item_sk#47, cr_order_number#48] -Join type: LeftOuter -Join condition: None +(50) CometSortMergeJoin +Left output [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] +Right output [4]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51] +Arguments: [cs_item_sk#41, cs_order_number#43], [cr_item_sk#48, cr_order_number#49], LeftOuter -(52) Project [codegen id : 19] -Output [8]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50] -Input [11]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] +(51) CometProject +Input [11]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46, cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51] +Arguments: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46, cr_return_amount#50, cr_net_loss#51], [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46, cr_return_amount#50, cr_net_loss#51] -(53) ReusedExchange [Reuses operator id: 125] -Output [1]: [d_date_sk#52] +(52) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#53] -(54) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_sold_date_sk#45] -Right keys [1]: [d_date_sk#52] -Join type: Inner -Join condition: None +(53) CometBroadcastHashJoin +Left output [8]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46, cr_return_amount#50, cr_net_loss#51] +Right output [1]: [d_date_sk#53] +Arguments: [cs_sold_date_sk#46], [d_date_sk#53], Inner, BuildRight -(55) Project [codegen id : 19] -Output [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50] -Input [9]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50, d_date_sk#52] +(54) CometProject +Input [9]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46, cr_return_amount#50, cr_net_loss#51, d_date_sk#53] +Arguments: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51], [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51] -(56) Scan parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +(55) Scan parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#54, cp_catalog_page_id#55] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(57) CometFilter -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Condition : isnotnull(cp_catalog_page_sk#53) - -(58) ColumnarToRow [codegen id : 16] -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] - -(59) BroadcastExchange -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -(60) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_catalog_page_sk#39] -Right keys [1]: [cp_catalog_page_sk#53] -Join type: Inner -Join condition: None - -(61) Project [codegen id : 19] -Output [7]: [cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] -Input [9]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_sk#53, cp_catalog_page_id#54] - -(62) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#55] - -(63) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_item_sk#40] -Right keys [1]: [i_item_sk#55] -Join type: Inner -Join condition: None - -(64) Project [codegen id : 19] -Output [6]: [cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] -Input [8]: [cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54, i_item_sk#55] - -(65) ReusedExchange [Reuses operator id: 34] -Output [1]: [p_promo_sk#56] - -(66) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_promo_sk#41] -Right keys [1]: [p_promo_sk#56] -Join type: Inner -Join condition: None - -(67) Project [codegen id : 19] -Output [5]: [cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] -Input [7]: [cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54, p_promo_sk#56] - -(68) HashAggregate [codegen id : 19] -Input [5]: [cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] -Keys [1]: [cp_catalog_page_id#54] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#43)), partial_sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))] -Aggregate Attributes [5]: [sum#57, sum#58, isEmpty#59, sum#60, isEmpty#61] -Results [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] - -(69) Exchange -Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] -Arguments: hashpartitioning(cp_catalog_page_id#54, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(70) HashAggregate [codegen id : 20] -Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] -Keys [1]: [cp_catalog_page_id#54] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#43)), sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00)), sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#43))#67, sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00))#68, sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))#69] -Results [5]: [catalog channel AS channel#70, concat(catalog_page, cp_catalog_page_id#54) AS id#71, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#43))#67,17,2) AS sales#72, sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00))#68 AS returns#73, sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))#69 AS profit#74] - -(71) Scan parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] +(56) CometFilter +Input [2]: [cp_catalog_page_sk#54, cp_catalog_page_id#55] +Condition : isnotnull(cp_catalog_page_sk#54) + +(57) CometBroadcastExchange +Input [2]: [cp_catalog_page_sk#54, cp_catalog_page_id#55] +Arguments: [cp_catalog_page_sk#54, cp_catalog_page_id#55] + +(58) CometBroadcastHashJoin +Left output [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51] +Right output [2]: [cp_catalog_page_sk#54, cp_catalog_page_id#55] +Arguments: [cs_catalog_page_sk#40], [cp_catalog_page_sk#54], Inner, BuildRight + +(59) CometProject +Input [9]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_sk#54, cp_catalog_page_id#55] +Arguments: [cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#55], [cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#55] + +(60) ReusedExchange [Reuses operator id: 26] +Output [1]: [i_item_sk#56] + +(61) CometBroadcastHashJoin +Left output [7]: [cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#55] +Right output [1]: [i_item_sk#56] +Arguments: [cs_item_sk#41], [i_item_sk#56], Inner, BuildRight + +(62) CometProject +Input [8]: [cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#55, i_item_sk#56] +Arguments: [cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#55], [cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#55] + +(63) ReusedExchange [Reuses operator id: 32] +Output [1]: [p_promo_sk#57] + +(64) CometBroadcastHashJoin +Left output [6]: [cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#55] +Right output [1]: [p_promo_sk#57] +Arguments: [cs_promo_sk#42], [p_promo_sk#57], Inner, BuildRight + +(65) CometProject +Input [7]: [cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#55, p_promo_sk#57] +Arguments: [cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#55], [cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#55] + +(66) ColumnarToRow [codegen id : 3] +Input [5]: [cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#55] + +(67) HashAggregate [codegen id : 3] +Input [5]: [cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#55] +Keys [1]: [cp_catalog_page_id#55] +Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#44)), partial_sum(coalesce(cast(cr_return_amount#50 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#45 - coalesce(cast(cr_net_loss#51 as decimal(12,2)), 0.00)))] +Aggregate Attributes [5]: [sum#58, sum#59, isEmpty#60, sum#61, isEmpty#62] +Results [6]: [cp_catalog_page_id#55, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] + +(68) RowToColumnar +Input [6]: [cp_catalog_page_id#55, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] + +(69) CometColumnarExchange +Input [6]: [cp_catalog_page_id#55, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] +Arguments: hashpartitioning(cp_catalog_page_id#55, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(70) ColumnarToRow [codegen id : 4] +Input [6]: [cp_catalog_page_id#55, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] + +(71) HashAggregate [codegen id : 4] +Input [6]: [cp_catalog_page_id#55, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] +Keys [1]: [cp_catalog_page_id#55] +Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#44)), sum(coalesce(cast(cr_return_amount#50 as decimal(12,2)), 0.00)), sum((cs_net_profit#45 - coalesce(cast(cr_net_loss#51 as decimal(12,2)), 0.00)))] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#44))#68, sum(coalesce(cast(cr_return_amount#50 as decimal(12,2)), 0.00))#69, sum((cs_net_profit#45 - coalesce(cast(cr_net_loss#51 as decimal(12,2)), 0.00)))#70] +Results [5]: [catalog channel AS channel#71, concat(catalog_page, cp_catalog_page_id#55) AS id#72, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#44))#68,17,2) AS sales#73, sum(coalesce(cast(cr_return_amount#50 as decimal(12,2)), 0.00))#69 AS returns#74, sum((cs_net_profit#45 - coalesce(cast(cr_net_loss#51 as decimal(12,2)), 0.00)))#70 AS profit#75] + +(72) Scan parquet spark_catalog.default.web_sales +Output [7]: [ws_item_sk#76, ws_web_site_sk#77, ws_promo_sk#78, ws_order_number#79, ws_ext_sales_price#80, ws_net_profit#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_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct -(72) CometFilter -Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Condition : ((isnotnull(ws_web_site_sk#76) AND isnotnull(ws_item_sk#75)) AND isnotnull(ws_promo_sk#77)) +(73) CometFilter +Input [7]: [ws_item_sk#76, ws_web_site_sk#77, ws_promo_sk#78, ws_order_number#79, ws_ext_sales_price#80, ws_net_profit#81, ws_sold_date_sk#82] +Condition : ((isnotnull(ws_web_site_sk#77) AND isnotnull(ws_item_sk#76)) AND isnotnull(ws_promo_sk#78)) -(73) ColumnarToRow [codegen id : 21] -Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] +(74) CometColumnarExchange +Input [7]: [ws_item_sk#76, ws_web_site_sk#77, ws_promo_sk#78, ws_order_number#79, ws_ext_sales_price#80, ws_net_profit#81, ws_sold_date_sk#82] +Arguments: hashpartitioning(ws_item_sk#76, ws_order_number#79, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(74) Exchange -Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Arguments: hashpartitioning(ws_item_sk#75, ws_order_number#78, 5), ENSURE_REQUIREMENTS, [plan_id=11] - -(75) Sort [codegen id : 22] -Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Arguments: [ws_item_sk#75 ASC NULLS FIRST, ws_order_number#78 ASC NULLS FIRST], false, 0 +(75) CometSort +Input [7]: [ws_item_sk#76, ws_web_site_sk#77, ws_promo_sk#78, ws_order_number#79, ws_ext_sales_price#80, ws_net_profit#81, ws_sold_date_sk#82] +Arguments: [ws_item_sk#76, ws_web_site_sk#77, ws_promo_sk#78, ws_order_number#79, ws_ext_sales_price#80, ws_net_profit#81, ws_sold_date_sk#82], [ws_item_sk#76 ASC NULLS FIRST, ws_order_number#79 ASC NULLS FIRST] (76) Scan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Output [5]: [wr_item_sk#84, wr_order_number#85, wr_return_amt#86, wr_net_loss#87, wr_returned_date_sk#88] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (77) CometFilter -Input [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] -Condition : (isnotnull(wr_item_sk#83) AND isnotnull(wr_order_number#84)) +Input [5]: [wr_item_sk#84, wr_order_number#85, wr_return_amt#86, wr_net_loss#87, wr_returned_date_sk#88] +Condition : (isnotnull(wr_item_sk#84) AND isnotnull(wr_order_number#85)) (78) CometProject -Input [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] -Arguments: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86], [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] - -(79) ColumnarToRow [codegen id : 23] -Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] +Input [5]: [wr_item_sk#84, wr_order_number#85, wr_return_amt#86, wr_net_loss#87, wr_returned_date_sk#88] +Arguments: [wr_item_sk#84, wr_order_number#85, wr_return_amt#86, wr_net_loss#87], [wr_item_sk#84, wr_order_number#85, wr_return_amt#86, wr_net_loss#87] -(80) Exchange -Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -Arguments: hashpartitioning(wr_item_sk#83, wr_order_number#84, 5), ENSURE_REQUIREMENTS, [plan_id=12] +(79) CometColumnarExchange +Input [4]: [wr_item_sk#84, wr_order_number#85, wr_return_amt#86, wr_net_loss#87] +Arguments: hashpartitioning(wr_item_sk#84, wr_order_number#85, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(81) Sort [codegen id : 24] -Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -Arguments: [wr_item_sk#83 ASC NULLS FIRST, wr_order_number#84 ASC NULLS FIRST], false, 0 +(80) CometSort +Input [4]: [wr_item_sk#84, wr_order_number#85, wr_return_amt#86, wr_net_loss#87] +Arguments: [wr_item_sk#84, wr_order_number#85, wr_return_amt#86, wr_net_loss#87], [wr_item_sk#84 ASC NULLS FIRST, wr_order_number#85 ASC NULLS FIRST] -(82) SortMergeJoin [codegen id : 29] -Left keys [2]: [ws_item_sk#75, ws_order_number#78] -Right keys [2]: [wr_item_sk#83, wr_order_number#84] -Join type: LeftOuter -Join condition: None +(81) CometSortMergeJoin +Left output [7]: [ws_item_sk#76, ws_web_site_sk#77, ws_promo_sk#78, ws_order_number#79, ws_ext_sales_price#80, ws_net_profit#81, ws_sold_date_sk#82] +Right output [4]: [wr_item_sk#84, wr_order_number#85, wr_return_amt#86, wr_net_loss#87] +Arguments: [ws_item_sk#76, ws_order_number#79], [wr_item_sk#84, wr_order_number#85], LeftOuter -(83) Project [codegen id : 29] -Output [8]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86] -Input [11]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] +(82) CometProject +Input [11]: [ws_item_sk#76, ws_web_site_sk#77, ws_promo_sk#78, ws_order_number#79, ws_ext_sales_price#80, ws_net_profit#81, ws_sold_date_sk#82, wr_item_sk#84, wr_order_number#85, wr_return_amt#86, wr_net_loss#87] +Arguments: [ws_item_sk#76, ws_web_site_sk#77, ws_promo_sk#78, ws_ext_sales_price#80, ws_net_profit#81, ws_sold_date_sk#82, wr_return_amt#86, wr_net_loss#87], [ws_item_sk#76, ws_web_site_sk#77, ws_promo_sk#78, ws_ext_sales_price#80, ws_net_profit#81, ws_sold_date_sk#82, wr_return_amt#86, wr_net_loss#87] -(84) ReusedExchange [Reuses operator id: 125] -Output [1]: [d_date_sk#88] +(83) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#89] -(85) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_sold_date_sk#81] -Right keys [1]: [d_date_sk#88] -Join type: Inner -Join condition: None +(84) CometBroadcastHashJoin +Left output [8]: [ws_item_sk#76, ws_web_site_sk#77, ws_promo_sk#78, ws_ext_sales_price#80, ws_net_profit#81, ws_sold_date_sk#82, wr_return_amt#86, wr_net_loss#87] +Right output [1]: [d_date_sk#89] +Arguments: [ws_sold_date_sk#82], [d_date_sk#89], Inner, BuildRight -(86) Project [codegen id : 29] -Output [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86] -Input [9]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86, d_date_sk#88] +(85) CometProject +Input [9]: [ws_item_sk#76, ws_web_site_sk#77, ws_promo_sk#78, ws_ext_sales_price#80, ws_net_profit#81, ws_sold_date_sk#82, wr_return_amt#86, wr_net_loss#87, d_date_sk#89] +Arguments: [ws_item_sk#76, ws_web_site_sk#77, ws_promo_sk#78, ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87], [ws_item_sk#76, ws_web_site_sk#77, ws_promo_sk#78, ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87] -(87) Scan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#89, web_site_id#90] +(86) Scan parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#90, web_site_id#91] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(88) CometFilter -Input [2]: [web_site_sk#89, web_site_id#90] -Condition : isnotnull(web_site_sk#89) - -(89) ColumnarToRow [codegen id : 26] -Input [2]: [web_site_sk#89, web_site_id#90] - -(90) BroadcastExchange -Input [2]: [web_site_sk#89, web_site_id#90] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] - -(91) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_web_site_sk#76] -Right keys [1]: [web_site_sk#89] -Join type: Inner -Join condition: None - -(92) Project [codegen id : 29] -Output [7]: [ws_item_sk#75, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] -Input [9]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_sk#89, web_site_id#90] - -(93) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#91] - -(94) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_item_sk#75] -Right keys [1]: [i_item_sk#91] -Join type: Inner -Join condition: None - -(95) Project [codegen id : 29] -Output [6]: [ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] -Input [8]: [ws_item_sk#75, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90, i_item_sk#91] - -(96) ReusedExchange [Reuses operator id: 34] -Output [1]: [p_promo_sk#92] - -(97) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_promo_sk#77] -Right keys [1]: [p_promo_sk#92] -Join type: Inner -Join condition: None - -(98) Project [codegen id : 29] -Output [5]: [ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] -Input [7]: [ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90, p_promo_sk#92] - -(99) HashAggregate [codegen id : 29] -Input [5]: [ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] -Keys [1]: [web_site_id#90] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#79)), partial_sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))] -Aggregate Attributes [5]: [sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] -Results [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] - -(100) Exchange -Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -Arguments: hashpartitioning(web_site_id#90, 5), ENSURE_REQUIREMENTS, [plan_id=14] - -(101) HashAggregate [codegen id : 30] -Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -Keys [1]: [web_site_id#90] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#79)), sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00)), sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#79))#103, sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00))#104, sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))#105] -Results [5]: [web channel AS channel#106, concat(web_site, web_site_id#90) AS id#107, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#79))#103,17,2) AS sales#108, sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00))#104 AS returns#109, sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))#105 AS profit#110] - -(102) Union - -(103) HashAggregate [codegen id : 31] -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#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116] -Results [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] - -(104) Exchange -Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] -Arguments: hashpartitioning(channel#34, id#35, 5), ENSURE_REQUIREMENTS, [plan_id=15] - -(105) HashAggregate [codegen id : 32] -Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] -Keys [2]: [channel#34, id#35] -Functions [3]: [sum(sales#36), sum(returns#37), sum(profit#38)] -Aggregate Attributes [3]: [sum(sales#36)#123, sum(returns#37)#124, sum(profit#38)#125] -Results [5]: [channel#34, id#35, cast(sum(sales#36)#123 as decimal(37,2)) AS sales#126, cast(sum(returns#37)#124 as decimal(38,2)) AS returns#127, cast(sum(profit#38)#125 as decimal(38,2)) AS profit#128] - -(106) ReusedExchange [Reuses operator id: 104] -Output [8]: [channel#129, id#130, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] - -(107) HashAggregate [codegen id : 64] -Input [8]: [channel#129, id#130, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] -Keys [2]: [channel#129, id#130] -Functions [3]: [sum(sales#137), sum(returns#138), sum(profit#139)] -Aggregate Attributes [3]: [sum(sales#137)#123, sum(returns#138)#124, sum(profit#139)#125] -Results [4]: [channel#129, sum(sales#137)#123 AS sales#140, sum(returns#138)#124 AS returns#141, sum(profit#139)#125 AS profit#142] - -(108) HashAggregate [codegen id : 64] -Input [4]: [channel#129, sales#140, returns#141, profit#142] -Keys [1]: [channel#129] -Functions [3]: [partial_sum(sales#140), partial_sum(returns#141), partial_sum(profit#142)] -Aggregate Attributes [6]: [sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] -Results [7]: [channel#129, sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154] - -(109) Exchange -Input [7]: [channel#129, sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154] -Arguments: hashpartitioning(channel#129, 5), ENSURE_REQUIREMENTS, [plan_id=16] - -(110) HashAggregate [codegen id : 65] -Input [7]: [channel#129, sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154] -Keys [1]: [channel#129] -Functions [3]: [sum(sales#140), sum(returns#141), sum(profit#142)] -Aggregate Attributes [3]: [sum(sales#140)#155, sum(returns#141)#156, sum(profit#142)#157] -Results [5]: [channel#129, null AS id#158, sum(sales#140)#155 AS sales#159, sum(returns#141)#156 AS returns#160, sum(profit#142)#157 AS profit#161] - -(111) ReusedExchange [Reuses operator id: 104] -Output [8]: [channel#162, id#163, sum#164, isEmpty#165, sum#166, isEmpty#167, sum#168, isEmpty#169] - -(112) HashAggregate [codegen id : 97] -Input [8]: [channel#162, id#163, sum#164, isEmpty#165, sum#166, isEmpty#167, sum#168, isEmpty#169] -Keys [2]: [channel#162, id#163] -Functions [3]: [sum(sales#170), sum(returns#171), sum(profit#172)] -Aggregate Attributes [3]: [sum(sales#170)#123, sum(returns#171)#124, sum(profit#172)#125] -Results [3]: [sum(sales#170)#123 AS sales#173, sum(returns#171)#124 AS returns#174, sum(profit#172)#125 AS profit#175] - -(113) HashAggregate [codegen id : 97] -Input [3]: [sales#173, returns#174, profit#175] +(87) CometFilter +Input [2]: [web_site_sk#90, web_site_id#91] +Condition : isnotnull(web_site_sk#90) + +(88) CometBroadcastExchange +Input [2]: [web_site_sk#90, web_site_id#91] +Arguments: [web_site_sk#90, web_site_id#91] + +(89) CometBroadcastHashJoin +Left output [7]: [ws_item_sk#76, ws_web_site_sk#77, ws_promo_sk#78, ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87] +Right output [2]: [web_site_sk#90, web_site_id#91] +Arguments: [ws_web_site_sk#77], [web_site_sk#90], Inner, BuildRight + +(90) CometProject +Input [9]: [ws_item_sk#76, ws_web_site_sk#77, ws_promo_sk#78, ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87, web_site_sk#90, web_site_id#91] +Arguments: [ws_item_sk#76, ws_promo_sk#78, ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87, web_site_id#91], [ws_item_sk#76, ws_promo_sk#78, ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87, web_site_id#91] + +(91) ReusedExchange [Reuses operator id: 26] +Output [1]: [i_item_sk#92] + +(92) CometBroadcastHashJoin +Left output [7]: [ws_item_sk#76, ws_promo_sk#78, ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87, web_site_id#91] +Right output [1]: [i_item_sk#92] +Arguments: [ws_item_sk#76], [i_item_sk#92], Inner, BuildRight + +(93) CometProject +Input [8]: [ws_item_sk#76, ws_promo_sk#78, ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87, web_site_id#91, i_item_sk#92] +Arguments: [ws_promo_sk#78, ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87, web_site_id#91], [ws_promo_sk#78, ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87, web_site_id#91] + +(94) ReusedExchange [Reuses operator id: 32] +Output [1]: [p_promo_sk#93] + +(95) CometBroadcastHashJoin +Left output [6]: [ws_promo_sk#78, ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87, web_site_id#91] +Right output [1]: [p_promo_sk#93] +Arguments: [ws_promo_sk#78], [p_promo_sk#93], Inner, BuildRight + +(96) CometProject +Input [7]: [ws_promo_sk#78, ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87, web_site_id#91, p_promo_sk#93] +Arguments: [ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87, web_site_id#91], [ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87, web_site_id#91] + +(97) ColumnarToRow [codegen id : 5] +Input [5]: [ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87, web_site_id#91] + +(98) HashAggregate [codegen id : 5] +Input [5]: [ws_ext_sales_price#80, ws_net_profit#81, wr_return_amt#86, wr_net_loss#87, web_site_id#91] +Keys [1]: [web_site_id#91] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#80)), partial_sum(coalesce(cast(wr_return_amt#86 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#81 - coalesce(cast(wr_net_loss#87 as decimal(12,2)), 0.00)))] +Aggregate Attributes [5]: [sum#94, sum#95, isEmpty#96, sum#97, isEmpty#98] +Results [6]: [web_site_id#91, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] + +(99) RowToColumnar +Input [6]: [web_site_id#91, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] + +(100) CometColumnarExchange +Input [6]: [web_site_id#91, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] +Arguments: hashpartitioning(web_site_id#91, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(101) ColumnarToRow [codegen id : 6] +Input [6]: [web_site_id#91, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] + +(102) HashAggregate [codegen id : 6] +Input [6]: [web_site_id#91, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] +Keys [1]: [web_site_id#91] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#80)), sum(coalesce(cast(wr_return_amt#86 as decimal(12,2)), 0.00)), sum((ws_net_profit#81 - coalesce(cast(wr_net_loss#87 as decimal(12,2)), 0.00)))] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#80))#104, sum(coalesce(cast(wr_return_amt#86 as decimal(12,2)), 0.00))#105, sum((ws_net_profit#81 - coalesce(cast(wr_net_loss#87 as decimal(12,2)), 0.00)))#106] +Results [5]: [web channel AS channel#107, concat(web_site, web_site_id#91) AS id#108, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#80))#104,17,2) AS sales#109, sum(coalesce(cast(wr_return_amt#86 as decimal(12,2)), 0.00))#105 AS returns#110, sum((ws_net_profit#81 - coalesce(cast(wr_net_loss#87 as decimal(12,2)), 0.00)))#106 AS profit#111] + +(103) Union + +(104) HashAggregate [codegen id : 7] +Input [5]: [channel#35, id#36, sales#37, returns#38, profit#39] +Keys [2]: [channel#35, id#36] +Functions [3]: [partial_sum(sales#37), partial_sum(returns#38), partial_sum(profit#39)] +Aggregate Attributes [6]: [sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] +Results [8]: [channel#35, id#36, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123] + +(105) RowToColumnar +Input [8]: [channel#35, id#36, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123] + +(106) CometColumnarExchange +Input [8]: [channel#35, id#36, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123] +Arguments: hashpartitioning(channel#35, id#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(107) ColumnarToRow [codegen id : 8] +Input [8]: [channel#35, id#36, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123] + +(108) HashAggregate [codegen id : 8] +Input [8]: [channel#35, id#36, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123] +Keys [2]: [channel#35, id#36] +Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] +Aggregate Attributes [3]: [sum(sales#37)#124, sum(returns#38)#125, sum(profit#39)#126] +Results [5]: [channel#35, id#36, cast(sum(sales#37)#124 as decimal(37,2)) AS sales#127, cast(sum(returns#38)#125 as decimal(38,2)) AS returns#128, cast(sum(profit#39)#126 as decimal(38,2)) AS profit#129] + +(109) ReusedExchange [Reuses operator id: 106] +Output [8]: [channel#130, id#131, sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] + +(110) ColumnarToRow [codegen id : 16] +Input [8]: [channel#130, id#131, sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] + +(111) HashAggregate [codegen id : 16] +Input [8]: [channel#130, id#131, sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] +Keys [2]: [channel#130, id#131] +Functions [3]: [sum(sales#138), sum(returns#139), sum(profit#140)] +Aggregate Attributes [3]: [sum(sales#138)#124, sum(returns#139)#125, sum(profit#140)#126] +Results [4]: [channel#130, sum(sales#138)#124 AS sales#141, sum(returns#139)#125 AS returns#142, sum(profit#140)#126 AS profit#143] + +(112) HashAggregate [codegen id : 16] +Input [4]: [channel#130, sales#141, returns#142, profit#143] +Keys [1]: [channel#130] +Functions [3]: [partial_sum(sales#141), partial_sum(returns#142), partial_sum(profit#143)] +Aggregate Attributes [6]: [sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] +Results [7]: [channel#130, sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155] + +(113) RowToColumnar +Input [7]: [channel#130, sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155] + +(114) CometColumnarExchange +Input [7]: [channel#130, sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155] +Arguments: hashpartitioning(channel#130, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] + +(115) ColumnarToRow [codegen id : 17] +Input [7]: [channel#130, sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155] + +(116) HashAggregate [codegen id : 17] +Input [7]: [channel#130, sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155] +Keys [1]: [channel#130] +Functions [3]: [sum(sales#141), sum(returns#142), sum(profit#143)] +Aggregate Attributes [3]: [sum(sales#141)#156, sum(returns#142)#157, sum(profit#143)#158] +Results [5]: [channel#130, null AS id#159, sum(sales#141)#156 AS sales#160, sum(returns#142)#157 AS returns#161, sum(profit#143)#158 AS profit#162] + +(117) ReusedExchange [Reuses operator id: 106] +Output [8]: [channel#163, id#164, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] + +(118) ColumnarToRow [codegen id : 25] +Input [8]: [channel#163, id#164, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] + +(119) HashAggregate [codegen id : 25] +Input [8]: [channel#163, id#164, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] +Keys [2]: [channel#163, id#164] +Functions [3]: [sum(sales#171), sum(returns#172), sum(profit#173)] +Aggregate Attributes [3]: [sum(sales#171)#124, sum(returns#172)#125, sum(profit#173)#126] +Results [3]: [sum(sales#171)#124 AS sales#174, sum(returns#172)#125 AS returns#175, sum(profit#173)#126 AS profit#176] + +(120) HashAggregate [codegen id : 25] +Input [3]: [sales#174, returns#175, profit#176] Keys: [] -Functions [3]: [partial_sum(sales#173), partial_sum(returns#174), partial_sum(profit#175)] -Aggregate Attributes [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] -Results [6]: [sum#182, isEmpty#183, sum#184, isEmpty#185, sum#186, isEmpty#187] +Functions [3]: [partial_sum(sales#174), partial_sum(returns#175), partial_sum(profit#176)] +Aggregate Attributes [6]: [sum#177, isEmpty#178, sum#179, isEmpty#180, sum#181, isEmpty#182] +Results [6]: [sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188] -(114) Exchange -Input [6]: [sum#182, isEmpty#183, sum#184, isEmpty#185, sum#186, isEmpty#187] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=17] +(121) RowToColumnar +Input [6]: [sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188] -(115) HashAggregate [codegen id : 98] -Input [6]: [sum#182, isEmpty#183, sum#184, isEmpty#185, sum#186, isEmpty#187] +(122) CometColumnarExchange +Input [6]: [sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] + +(123) ColumnarToRow [codegen id : 26] +Input [6]: [sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188] + +(124) HashAggregate [codegen id : 26] +Input [6]: [sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188] Keys: [] -Functions [3]: [sum(sales#173), sum(returns#174), sum(profit#175)] -Aggregate Attributes [3]: [sum(sales#173)#188, sum(returns#174)#189, sum(profit#175)#190] -Results [5]: [null AS channel#191, null AS id#192, sum(sales#173)#188 AS sales#193, sum(returns#174)#189 AS returns#194, sum(profit#175)#190 AS profit#195] +Functions [3]: [sum(sales#174), sum(returns#175), sum(profit#176)] +Aggregate Attributes [3]: [sum(sales#174)#189, sum(returns#175)#190, sum(profit#176)#191] +Results [5]: [null AS channel#192, null AS id#193, sum(sales#174)#189 AS sales#194, sum(returns#175)#190 AS returns#195, sum(profit#176)#191 AS profit#196] -(116) Union +(125) Union -(117) HashAggregate [codegen id : 99] -Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] -Keys [5]: [channel#34, id#35, sales#126, returns#127, profit#128] +(126) HashAggregate [codegen id : 27] +Input [5]: [channel#35, id#36, sales#127, returns#128, profit#129] +Keys [5]: [channel#35, id#36, sales#127, returns#128, profit#129] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#34, id#35, sales#126, returns#127, profit#128] +Results [5]: [channel#35, id#36, sales#127, returns#128, profit#129] + +(127) RowToColumnar +Input [5]: [channel#35, id#36, sales#127, returns#128, profit#129] -(118) Exchange -Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] -Arguments: hashpartitioning(channel#34, id#35, sales#126, returns#127, profit#128, 5), ENSURE_REQUIREMENTS, [plan_id=18] +(128) CometColumnarExchange +Input [5]: [channel#35, id#36, sales#127, returns#128, profit#129] +Arguments: hashpartitioning(channel#35, id#36, sales#127, returns#128, profit#129, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(119) HashAggregate [codegen id : 100] -Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] -Keys [5]: [channel#34, id#35, sales#126, returns#127, profit#128] +(129) CometHashAggregate +Input [5]: [channel#35, id#36, sales#127, returns#128, profit#129] +Keys [5]: [channel#35, id#36, sales#127, returns#128, profit#129] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#34, id#35, sales#126, returns#127, profit#128] -(120) TakeOrderedAndProject -Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] -Arguments: 100, [channel#34 ASC NULLS FIRST, id#35 ASC NULLS FIRST], [channel#34, id#35, sales#126, returns#127, profit#128] +(130) CometTakeOrderedAndProject +Input [5]: [channel#35, id#36, sales#127, returns#128, profit#129] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#35 ASC NULLS FIRST,id#36 ASC NULLS FIRST], output=[channel#35,id#36,sales#127,returns#128,profit#129]), [channel#35, id#36, sales#127, returns#128, profit#129], 100, [channel#35 ASC NULLS FIRST, id#36 ASC NULLS FIRST], [channel#35, id#36, sales#127, returns#128, profit#129] + +(131) ColumnarToRow [codegen id : 28] +Input [5]: [channel#35, id#36, sales#127, returns#128, profit#129] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (125) -+- * ColumnarToRow (124) - +- CometProject (123) - +- CometFilter (122) - +- CometScan parquet spark_catalog.default.date_dim (121) +BroadcastExchange (136) ++- * ColumnarToRow (135) + +- CometProject (134) + +- CometFilter (133) + +- CometScan parquet spark_catalog.default.date_dim (132) -(121) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#196] +(132) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#15] 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 -(122) CometFilter -Input [2]: [d_date_sk#14, d_date#196] -Condition : (((isnotnull(d_date#196) AND (d_date#196 >= 1998-08-04)) AND (d_date#196 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) +(133) CometFilter +Input [2]: [d_date_sk#14, d_date#15] +Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) -(123) CometProject -Input [2]: [d_date_sk#14, d_date#196] +(134) CometProject +Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(124) ColumnarToRow [codegen id : 1] +(135) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(125) BroadcastExchange +(136) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] -Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 41 Hosting Expression = cs_sold_date_sk#46 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 71 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 72 Hosting Expression = ws_sold_date_sk#82 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/simplified.txt index 0793067df..8ad016ab7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/simplified.txt @@ -1,207 +1,169 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (100) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Exchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (99) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (32) - 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 (31) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (10) - HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] +WholeStageCodegen (28) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [channel,id,sales,returns,profit] + CometColumnarExchange [channel,id,sales,returns,profit] #1 + RowToColumnar + WholeStageCodegen (27) + HashAggregate [channel,id,sales,returns,profit] + InputAdapter + Union + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel,id] #2 + RowToColumnar + WholeStageCodegen (7) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange [s_store_id] #3 - WholeStageCodegen (9) - HashAggregate [s_store_id,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - SortMergeJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - WholeStageCodegen (2) - Sort [ss_item_sk,ss_ticket_number] - InputAdapter - Exchange [ss_item_sk,ss_ticket_number] #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - WholeStageCodegen (4) - Sort [sr_item_sk,sr_ticket_number] - InputAdapter - Exchange [sr_item_sk,sr_ticket_number] #6 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_id] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - WholeStageCodegen (20) - HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [cp_catalog_page_id] #10 - WholeStageCodegen (19) - HashAggregate [cp_catalog_page_id,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_catalog_page_sk,cp_catalog_page_sk] - Project [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] - InputAdapter - WholeStageCodegen (12) - Sort [cs_item_sk,cs_order_number] - InputAdapter - Exchange [cs_item_sk,cs_order_number] #11 - WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - WholeStageCodegen (14) - Sort [cr_item_sk,cr_order_number] - InputAdapter - Exchange [cr_item_sk,cr_order_number] #12 - WholeStageCodegen (13) - ColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - InputAdapter - ReusedExchange [i_item_sk] #8 - InputAdapter - ReusedExchange [p_promo_sk] #9 - WholeStageCodegen (30) - HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [web_site_id] #14 - WholeStageCodegen (29) - HashAggregate [web_site_id,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_promo_sk,p_promo_sk] - Project [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - SortMergeJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] - InputAdapter - WholeStageCodegen (22) - Sort [ws_item_sk,ws_order_number] - InputAdapter - Exchange [ws_item_sk,ws_order_number] #15 - WholeStageCodegen (21) - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - WholeStageCodegen (24) - Sort [wr_item_sk,wr_order_number] - InputAdapter - Exchange [wr_item_sk,wr_order_number] #16 - WholeStageCodegen (23) - ColumnarToRow - InputAdapter - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (26) - ColumnarToRow - InputAdapter - CometFilter [web_site_sk,web_site_id] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - InputAdapter - ReusedExchange [i_item_sk] #8 - InputAdapter - ReusedExchange [p_promo_sk] #9 - WholeStageCodegen (65) - 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] #18 - WholeStageCodegen (64) - 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 (98) - 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 #19 - WholeStageCodegen (97) - 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 - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + Union + WholeStageCodegen (2) + HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [s_store_id] #3 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [s_store_id,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] + CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] + CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] + CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk,ss_ticket_number] #4 + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometColumnarExchange [sr_item_sk,sr_ticket_number] #6 + CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #7 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [s_store_sk,s_store_id] #8 + CometFilter [s_store_sk,s_store_id] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange [i_item_sk] #9 + CometProject [i_item_sk] + CometFilter [i_item_sk,i_current_price] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] + CometBroadcastExchange [p_promo_sk] #10 + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] + CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] + WholeStageCodegen (4) + HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [cp_catalog_page_id] #11 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [cp_catalog_page_id,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] + CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] + CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] + CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometColumnarExchange [cs_item_sk,cs_order_number] #12 + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometColumnarExchange [cr_item_sk,cr_order_number] #13 + CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #14 + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + ReusedExchange [i_item_sk] #9 + ReusedExchange [p_promo_sk] #10 + WholeStageCodegen (6) + HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [web_site_id] #15 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [web_site_id,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] + CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] + CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] + CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometColumnarExchange [ws_item_sk,ws_order_number] #16 + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometColumnarExchange [wr_item_sk,wr_order_number] #17 + CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange [web_site_sk,web_site_id] #18 + CometFilter [web_site_sk,web_site_id] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + ReusedExchange [i_item_sk] #9 + ReusedExchange [p_promo_sk] #10 + WholeStageCodegen (17) + HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel] #19 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + WholeStageCodegen (26) + HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange #20 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 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 efdca9dca..9d890a1a9 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,40 +1,50 @@ == Physical Plan == -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) +TakeOrderedAndProject (46) ++- * Project (45) + +- Window (44) + +- * ColumnarToRow (43) + +- CometSort (42) + +- CometColumnarExchange (41) + +- CometHashAggregate (40) + +- CometColumnarExchange (39) + +- RowToColumnar (38) + +- * HashAggregate (37) + +- Union (36) + :- * HashAggregate (19) + : +- * ColumnarToRow (18) + : +- CometColumnarExchange (17) + : +- RowToColumnar (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 (27) + : +- * ColumnarToRow (26) + : +- CometColumnarExchange (25) + : +- RowToColumnar (24) + : +- * HashAggregate (23) + : +- * HashAggregate (22) + : +- * ColumnarToRow (21) + : +- ReusedExchange (20) + +- * HashAggregate (35) + +- * ColumnarToRow (34) + +- CometColumnarExchange (33) + +- RowToColumnar (32) + +- * HashAggregate (31) + +- * HashAggregate (30) + +- * ColumnarToRow (29) + +- ReusedExchange (28) (1) Scan parquet spark_catalog.default.web_sales @@ -111,142 +121,170 @@ Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] Aggregate Attributes [1]: [sum#10] Results [3]: [i_category#9, i_class#8, sum#11] -(16) Exchange +(16) RowToColumnar Input [3]: [i_category#9, i_class#8, sum#11] -Arguments: hashpartitioning(i_category#9, i_class#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(17) HashAggregate [codegen id : 2] +(17) CometColumnarExchange +Input [3]: [i_category#9, i_class#8, sum#11] +Arguments: hashpartitioning(i_category#9, i_class#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(18) ColumnarToRow [codegen id : 2] +Input [3]: [i_category#9, i_class#8, sum#11] + +(19) 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))#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] +(20) ReusedExchange [Reuses operator id: 17] Output [3]: [i_category#17, i_class#18, sum#19] -(19) HashAggregate [codegen id : 4] +(21) ColumnarToRow [codegen id : 4] +Input [3]: [i_category#17, i_class#18, sum#19] + +(22) 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] +(23) 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 +(24) RowToColumnar 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] +(25) CometColumnarExchange +Input [3]: [i_category#17, sum#24, isEmpty#25] +Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(26) ColumnarToRow [codegen id : 5] +Input [3]: [i_category#17, sum#24, isEmpty#25] + +(27) 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] +(28) ReusedExchange [Reuses operator id: 17] Output [3]: [i_category#32, i_class#33, sum#34] -(24) HashAggregate [codegen id : 7] +(29) ColumnarToRow [codegen id : 7] +Input [3]: [i_category#32, i_class#33, sum#34] + +(30) 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] +(31) HashAggregate [codegen id : 7] Input [1]: [total_sum#36] Keys: [] Functions [1]: [partial_sum(total_sum#36)] Aggregate Attributes [2]: [sum#37, isEmpty#38] Results [2]: [sum#39, isEmpty#40] -(26) Exchange +(32) RowToColumnar +Input [2]: [sum#39, isEmpty#40] + +(33) CometColumnarExchange +Input [2]: [sum#39, isEmpty#40] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(34) ColumnarToRow [codegen id : 8] Input [2]: [sum#39, isEmpty#40] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] -(27) HashAggregate [codegen id : 8] +(35) HashAggregate [codegen id : 8] Input [2]: [sum#39, isEmpty#40] Keys: [] 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] -(28) Union +(36) Union -(29) HashAggregate [codegen id : 9] +(37) 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#13, i_category#9, i_class#8, g_category#14, g_class#15, lochierarchy#16] -(30) Exchange +(38) RowToColumnar 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] -(31) HashAggregate [codegen id : 10] +(39) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=4] + +(40) CometHashAggregate 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#13, i_category#9, i_class#8, lochierarchy#16, CASE WHEN (g_class#15 = 0) THEN i_category#9 END AS _w0#48] -(32) Exchange +(41) CometColumnarExchange +Input [5]: [total_sum#13, i_category#9, i_class#8, lochierarchy#16, _w0#48] +Arguments: hashpartitioning(lochierarchy#16, _w0#48, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(42) CometSort 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] +Arguments: [total_sum#13, i_category#9, i_class#8, lochierarchy#16, _w0#48], [lochierarchy#16 ASC NULLS FIRST, _w0#48 ASC NULLS FIRST, total_sum#13 DESC NULLS LAST] -(33) Sort [codegen id : 11] +(43) ColumnarToRow [codegen id : 10] 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 -(34) Window +(44) 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] -(35) Project [codegen id : 12] +(45) Project [codegen id : 11] 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] -(36) TakeOrderedAndProject +(46) 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 (41) -+- * ColumnarToRow (40) - +- CometProject (39) - +- CometFilter (38) - +- CometScan parquet spark_catalog.default.date_dim (37) +BroadcastExchange (51) ++- * ColumnarToRow (50) + +- CometProject (49) + +- CometFilter (48) + +- CometScan parquet spark_catalog.default.date_dim (47) -(37) Scan parquet spark_catalog.default.date_dim +(47) 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 -(38) CometFilter +(48) 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)) -(39) CometProject +(49) CometProject Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(40) ColumnarToRow [codegen id : 1] +(50) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(41) BroadcastExchange +(51) BroadcastExchange Input [1]: [d_date_sk#5] 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 8996119f2..66385580c 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,64 +1,72 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (12) + WholeStageCodegen (11) Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (11) - Sort [lochierarchy,_w0,total_sum] + WholeStageCodegen (10) + ColumnarToRow InputAdapter - Exchange [lochierarchy,_w0] #1 - 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 + CometSort [total_sum,i_category,i_class,lochierarchy,_w0] + CometColumnarExchange [lochierarchy,_w0] #1 + CometHashAggregate [total_sum,i_category,i_class,lochierarchy,_w0,g_category,g_class] + CometColumnarExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 + RowToColumnar WholeStageCodegen (9) HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] InputAdapter Union 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 (1) - HashAggregate [i_category,i_class,ws_net_paid] [sum,sum] - ColumnarToRow - InputAdapter - CometProject [ws_net_paid,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometFilter [i_item_sk,i_class,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class] #3 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [i_category,i_class,ws_net_paid] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [ws_net_paid,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] + CometProject [ws_item_sk,ws_net_paid] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometFilter [i_item_sk,i_class,i_category] + 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] #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 + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category] #7 + RowToColumnar + 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] + ColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,sum] #3 WholeStageCodegen (8) HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty] - InputAdapter - 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 - ReusedExchange [i_category,i_class,sum] #3 + ColumnarToRow + InputAdapter + CometColumnarExchange #8 + RowToColumnar + WholeStageCodegen (7) + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] + ColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,sum] #3 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 1b71bb0ca..5b3adf06d 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,27 +1,33 @@ == Physical Plan == -* 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) +* ColumnarToRow (29) ++- CometSort (28) + +- CometColumnarExchange (27) + +- RowToColumnar (26) + +- * Project (25) + +- Window (24) + +- * ColumnarToRow (23) + +- CometSort (22) + +- CometColumnarExchange (21) + +- RowToColumnar (20) + +- * HashAggregate (19) + +- * ColumnarToRow (18) + +- CometColumnarExchange (17) + +- RowToColumnar (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 @@ -98,70 +104,88 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] 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] -(16) Exchange +(16) RowToColumnar 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] -(17) HashAggregate [codegen id : 2] +(17) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=1] + +(18) ColumnarToRow [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] + +(19) 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))#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] -(18) Exchange +(20) RowToColumnar +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17] + +(21) CometColumnarExchange 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] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(19) Sort [codegen id : 3] +(22) CometSort 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 +Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17], [i_class#9 ASC NULLS FIRST] -(20) Window +(23) ColumnarToRow [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] + +(24) 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] -(21) Project [codegen id : 4] +(25) 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] -(22) Exchange +(26) RowToColumnar +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, revenueratio#19] + +(27) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=3] + +(28) CometSort 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] +Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, revenueratio#19], [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] -(23) Sort [codegen id : 5] +(29) ColumnarToRow [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 (28) -+- * ColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan parquet spark_catalog.default.date_dim (24) +BroadcastExchange (34) ++- * ColumnarToRow (33) + +- CometProject (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.date_dim (30) -(24) Scan parquet spark_catalog.default.date_dim +(30) 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 -(25) CometFilter +(31) 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)) -(26) CometProject +(32) CometProject Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(27) ColumnarToRow [codegen id : 1] +(33) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(28) BroadcastExchange +(34) BroadcastExchange Input [1]: [d_date_sk#11] 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 3fc9eb261..dd87e6394 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,41 +1,47 @@ WholeStageCodegen (5) - Sort [i_category,i_class,i_item_id,i_item_desc,revenueratio] + ColumnarToRow InputAdapter - Exchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - 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 (3) - Sort [i_class] - InputAdapter - Exchange [i_class] #2 - 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 (1) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 + RowToColumnar + 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 (3) + ColumnarToRow + InputAdapter + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometColumnarExchange [i_class] #2 + RowToColumnar + 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] ColumnarToRow InputAdapter - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_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 #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_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 #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + 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 43d59eb84..f2de73f90 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,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (39) -+- * HashAggregate (38) - +- Exchange (37) - +- * ColumnarToRow (36) +* ColumnarToRow (39) ++- CometTakeOrderedAndProject (38) + +- CometHashAggregate (37) + +- CometColumnarExchange (36) +- CometHashAggregate (35) +- CometProject (34) +- CometBroadcastHashJoin (33) @@ -207,23 +207,21 @@ Input [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purch 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)] -(36) ColumnarToRow [codegen id : 1] +(36) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(37) CometHashAggregate 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)#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] -(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] +(38) CometTakeOrderedAndProject +Input [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#32, cd_purchase_estimate#26, cnt2#33, cd_credit_rating#27, cnt3#34, cd_dep_count#28, cnt4#35, cd_dep_employed_count#29, cnt5#36, cd_dep_college_count#30, cnt6#37] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[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], output=[cd_gender#23,cd_marital_status#24,cd_education_status#25,cnt1#32,cd_purchase_estimate#26,cnt2#33,cd_credit_rating#27,cnt3#34,cd_dep_count#28,cnt4#35,cd_dep_employed_count#29,cnt5#36,cd_dep_college_count#30,cnt6#37]), [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#32, cd_purchase_estimate#26, cnt2#33, cd_credit_rating#27, cnt3#34, cd_dep_count#28, cnt4#35, cd_dep_employed_count#29, cnt5#36, cd_dep_college_count#30, cnt6#37], 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#32, cd_purchase_estimate#26, cnt2#33, cd_credit_rating#27, cnt3#34, cd_dep_count#28, cnt4#35, cd_dep_employed_count#29, cnt5#36, cd_dep_college_count#30, cnt6#37] + +(39) ColumnarToRow [codegen id : 1] +Input [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#32, cd_purchase_estimate#26, cnt2#33, cd_credit_rating#27, cnt3#34, cd_dep_count#28, cnt4#35, cd_dep_employed_count#29, cnt5#36, cd_dep_college_count#30, cnt6#37] ===== Subqueries ===== 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 22c208f2a..243bdbb91 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,53 +1,51 @@ -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 (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 (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,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,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 [c_current_cdemo_sk] - CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk] - CometProject [c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customer_sk] - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] - CometFilter [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] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [customer_sk] #5 - CometUnion [customer_sk] - CometProject [ws_bill_customer_sk] [customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,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_ship_customer_sk,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 [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - CometBroadcastExchange [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] #7 - CometFilter [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] - 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] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cnt1,cd_purchase_estimate,cnt2,cd_credit_rating,cnt3,cd_dep_count,cnt4,cd_dep_employed_count,cnt5,cd_dep_college_count,cnt6] + CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cnt1,cd_purchase_estimate,cnt2,cd_credit_rating,cnt3,cd_dep_count,cnt4,cd_dep_employed_count,cnt5,cd_dep_college_count,cnt6,count,count(1)] + CometColumnarExchange [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 + 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,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,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 [c_current_cdemo_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk] + CometProject [c_current_cdemo_sk,c_current_addr_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customer_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [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] + CometBroadcastExchange [ss_customer_sk] #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [customer_sk] #5 + CometUnion [customer_sk] + CometProject [ws_bill_customer_sk] [customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,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_ship_customer_sk,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 [ca_address_sk] #6 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_county] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + CometBroadcastExchange [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] #7 + CometFilter [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] + 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 f2b239def..0ae630c49 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 @@ -1,75 +1,72 @@ == Physical Plan == -TakeOrderedAndProject (71) -+- * Project (70) - +- * BroadcastHashJoin Inner BuildRight (69) - :- * 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) - +- * 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) +* ColumnarToRow (68) ++- CometTakeOrderedAndProject (67) + +- CometProject (66) + +- CometBroadcastHashJoin (65) + :- CometProject (50) + : +- CometBroadcastHashJoin (49) + : :- CometBroadcastHashJoin (33) + : : :- CometFilter (16) + : : : +- CometHashAggregate (15) + : : : +- CometColumnarExchange (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) + : : +- CometBroadcastExchange (32) + : : +- CometHashAggregate (31) + : : +- CometColumnarExchange (30) + : : +- CometHashAggregate (29) + : : +- CometProject (28) + : : +- CometBroadcastHashJoin (27) + : : :- CometProject (23) + : : : +- CometBroadcastHashJoin (22) + : : : :- CometFilter (18) + : : : : +- CometScan parquet spark_catalog.default.customer (17) + : : : +- CometBroadcastExchange (21) + : : : +- CometFilter (20) + : : : +- CometScan parquet spark_catalog.default.store_sales (19) + : : +- CometBroadcastExchange (26) + : : +- CometFilter (25) + : : +- CometScan parquet spark_catalog.default.date_dim (24) + : +- CometBroadcastExchange (48) + : +- CometFilter (47) + : +- CometHashAggregate (46) + : +- CometColumnarExchange (45) + : +- CometHashAggregate (44) + : +- CometProject (43) + : +- CometBroadcastHashJoin (42) + : :- CometProject (40) + : : +- CometBroadcastHashJoin (39) + : : :- CometFilter (35) + : : : +- CometScan parquet spark_catalog.default.customer (34) + : : +- CometBroadcastExchange (38) + : : +- CometFilter (37) + : : +- CometScan parquet spark_catalog.default.web_sales (36) + : +- ReusedExchange (41) + +- CometBroadcastExchange (64) + +- CometHashAggregate (63) + +- CometColumnarExchange (62) + +- CometHashAggregate (61) + +- CometProject (60) + +- CometBroadcastHashJoin (59) + :- CometProject (57) + : +- CometBroadcastHashJoin (56) + : :- CometFilter (52) + : : +- CometScan parquet spark_catalog.default.customer (51) + : +- CometBroadcastExchange (55) + : +- CometFilter (54) + : +- CometScan parquet spark_catalog.default.web_sales (53) + +- ReusedExchange (58) (1) Scan parquet spark_catalog.default.customer @@ -137,337 +134,317 @@ Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_fl 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)))] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(16) HashAggregate [codegen id : 8] +(15) CometHashAggregate 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)))#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] -(17) Filter [codegen id : 8] -Input [2]: [customer_id#18, year_total#19] -Condition : (isnotnull(year_total#19) AND (year_total#19 > 0.00)) +(16) CometFilter +Input [2]: [customer_id#17, year_total#18] +Condition : (isnotnull(year_total#18) AND (year_total#18 > 0.00)) -(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] +(17) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(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)) +(18) CometFilter +Input [8]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26] +Condition : (isnotnull(c_customer_sk#19) AND isnotnull(c_customer_id#20)) -(20) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] +(19) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_customer_sk#27, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#31), dynamicpruningexpression(ss_sold_date_sk#31 IN dynamicpruning#32)] +PartitionFilters: [isnotnull(ss_sold_date_sk#30), dynamicpruningexpression(ss_sold_date_sk#30 IN dynamicpruning#31)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(21) CometFilter -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) +(20) CometFilter +Input [4]: [ss_customer_sk#27, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30] +Condition : isnotnull(ss_customer_sk#27) -(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] +(21) CometBroadcastExchange +Input [4]: [ss_customer_sk#27, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30] +Arguments: [ss_customer_sk#27, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30] -(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 +(22) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26] +Right output [4]: [ss_customer_sk#27, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30] +Arguments: [c_customer_sk#19], [ss_customer_sk#27], Inner, BuildRight -(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] +(23) CometProject +Input [12]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, ss_customer_sk#27, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30] +Arguments: [c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30], [c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30] -(25) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_year#34] +(24) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#32, d_year#33] 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 -(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 - -(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] +(25) CometFilter +Input [2]: [d_date_sk#32, d_year#33] +Condition : ((isnotnull(d_year#33) AND (d_year#33 = 2002)) AND isnotnull(d_date_sk#32)) + +(26) CometBroadcastExchange +Input [2]: [d_date_sk#32, d_year#33] +Arguments: [d_date_sk#32, d_year#33] + +(27) CometBroadcastHashJoin +Left output [10]: [c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30] +Right output [2]: [d_date_sk#32, d_year#33] +Arguments: [ss_sold_date_sk#30], [d_date_sk#32], Inner, BuildRight + +(28) CometProject +Input [12]: [c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, ss_ext_discount_amt#28, ss_ext_list_price#29, ss_sold_date_sk#30, d_date_sk#32, d_year#33] +Arguments: [c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, ss_ext_discount_amt#28, ss_ext_list_price#29, d_year#33], [c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, ss_ext_discount_amt#28, ss_ext_list_price#29, d_year#33] + +(29) CometHashAggregate +Input [10]: [c_customer_id#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, ss_ext_discount_amt#28, ss_ext_list_price#29, d_year#33] +Keys [8]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#33, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26] +Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#29 - ss_ext_discount_amt#28)))] + +(30) CometColumnarExchange +Input [9]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#33, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, sum#34] +Arguments: hashpartitioning(c_customer_id#20, c_first_name#21, c_last_name#22, d_year#33, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(31) CometHashAggregate +Input [9]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#33, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26, sum#34] +Keys [8]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#33, c_preferred_cust_flag#23, c_birth_country#24, c_login#25, c_email_address#26] +Functions [1]: [sum(UnscaledValue((ss_ext_list_price#29 - ss_ext_discount_amt#28)))] + +(32) CometBroadcastExchange +Input [5]: [customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38, year_total#39] +Arguments: [customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38, year_total#39] + +(33) CometBroadcastHashJoin +Left output [2]: [customer_id#17, year_total#18] +Right output [5]: [customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38, year_total#39] +Arguments: [customer_id#17], [customer_id#35], Inner, BuildRight + +(34) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(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)) +(35) CometFilter +Input [8]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47] +Condition : (isnotnull(c_customer_sk#40) AND isnotnull(c_customer_id#41)) -(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] +(36) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_bill_customer_sk#48, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51] 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#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#52)] 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) +(37) CometFilter +Input [4]: [ws_bill_customer_sk#48, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51] +Condition : isnotnull(ws_bill_customer_sk#48) + +(38) CometBroadcastExchange +Input [4]: [ws_bill_customer_sk#48, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51] +Arguments: [ws_bill_customer_sk#48, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51] -(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] +(39) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47] +Right output [4]: [ws_bill_customer_sk#48, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51] +Arguments: [c_customer_sk#40], [ws_bill_customer_sk#48], Inner, BuildRight -(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 +(40) CometProject +Input [12]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, ws_bill_customer_sk#48, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51] +Arguments: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51], [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51] -(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] +(41) ReusedExchange [Reuses operator id: 10] +Output [2]: [d_date_sk#53, d_year#54] -(43) ReusedExchange [Reuses operator id: 10] -Output [2]: [d_date_sk#54, d_year#55] +(42) CometBroadcastHashJoin +Left output [10]: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51] +Right output [2]: [d_date_sk#53, d_year#54] +Arguments: [ws_sold_date_sk#51], [d_date_sk#53], Inner, BuildRight -(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 +(43) CometProject +Input [12]: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51, d_date_sk#53, d_year#54] +Arguments: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, ws_ext_discount_amt#49, ws_ext_list_price#50, d_year#54], [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, ws_ext_discount_amt#49, ws_ext_list_price#50, d_year#54] -(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] +(44) CometHashAggregate +Input [10]: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, ws_ext_discount_amt#49, ws_ext_list_price#50, d_year#54] +Keys [8]: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, d_year#54] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#50 - ws_ext_discount_amt#49)))] + +(45) CometColumnarExchange +Input [9]: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, d_year#54, sum#55] +Arguments: hashpartitioning(c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, d_year#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (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] - -(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 - -(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] - -(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] +Input [9]: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, d_year#54, sum#55] +Keys [8]: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, d_year#54] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#50 - ws_ext_discount_amt#49)))] + +(47) CometFilter +Input [2]: [customer_id#56, year_total#57] +Condition : (isnotnull(year_total#57) AND (year_total#57 > 0.00)) + +(48) CometBroadcastExchange +Input [2]: [customer_id#56, year_total#57] +Arguments: [customer_id#56, year_total#57] + +(49) CometBroadcastHashJoin +Left output [7]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38, year_total#39] +Right output [2]: [customer_id#56, year_total#57] +Arguments: [customer_id#17], [customer_id#56], Inner, BuildRight + +(50) CometProject +Input [9]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38, year_total#39, customer_id#56, year_total#57] +Arguments: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38, year_total#39, year_total#57], [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38, year_total#39, year_total#57] + +(51) 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#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)) +(52) 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)) -(56) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#68, ws_ext_discount_amt#69, ws_ext_list_price#70, ws_sold_date_sk#71] +(53) Scan parquet spark_catalog.default.web_sales +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#71), dynamicpruningexpression(ws_sold_date_sk#71 IN dynamicpruning#72)] +PartitionFilters: [isnotnull(ws_sold_date_sk#69), dynamicpruningexpression(ws_sold_date_sk#69 IN dynamicpruning#70)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(57) CometFilter -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) +(54) CometFilter +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) -(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] +(55) 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] + +(56) 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 + +(57) 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] + +(58) ReusedExchange [Reuses operator id: 26] +Output [2]: [d_date_sk#71, d_year#72] (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 +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 (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) ReusedExchange [Reuses operator id: 27] -Output [2]: [d_date_sk#73, d_year#74] - -(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) 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) 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) 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#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 : 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#76, year_total#77] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] - -(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#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 : 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#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] +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] + +(61) 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)))] + +(62) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=4] + +(63) CometHashAggregate +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)))] + +(64) CometBroadcastExchange +Input [2]: [customer_id#74, year_total#75] +Arguments: [customer_id#74, year_total#75] + +(65) CometBroadcastHashJoin +Left output [8]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38, year_total#39, year_total#57] +Right output [2]: [customer_id#74, year_total#75] +Arguments: [customer_id#17], [customer_id#74], Inner, (CASE WHEN (year_total#57 > 0.00) THEN (year_total#75 / year_total#57) ELSE 0E-20 END > CASE WHEN (year_total#18 > 0.00) THEN (year_total#39 / year_total#18) ELSE 0E-20 END), BuildRight + +(66) CometProject +Input [10]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38, year_total#39, year_total#57, customer_id#74, year_total#75] +Arguments: [customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38], [customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38] + +(67) CometTakeOrderedAndProject +Input [4]: [customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#35 ASC NULLS FIRST,customer_first_name#36 ASC NULLS FIRST,customer_last_name#37 ASC NULLS FIRST,customer_email_address#38 ASC NULLS FIRST], output=[customer_id#35,customer_first_name#36,customer_last_name#37,customer_email_address#38]), [customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38], 100, [customer_id#35 ASC NULLS FIRST, customer_first_name#36 ASC NULLS FIRST, customer_last_name#37 ASC NULLS FIRST, customer_email_address#38 ASC NULLS FIRST], [customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38] + +(68) ColumnarToRow [codegen id : 1] +Input [4]: [customer_id#35, customer_first_name#36, customer_last_name#37, customer_email_address#38] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (75) -+- * ColumnarToRow (74) - +- CometFilter (73) - +- CometScan parquet spark_catalog.default.date_dim (72) +BroadcastExchange (72) ++- * ColumnarToRow (71) + +- CometFilter (70) + +- CometScan parquet spark_catalog.default.date_dim (69) -(72) Scan parquet spark_catalog.default.date_dim +(69) 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 -(73) CometFilter +(70) 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)) -(74) ColumnarToRow [codegen id : 1] +(71) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#14, d_year#15] -(75) BroadcastExchange +(72) BroadcastExchange Input [2]: [d_date_sk#14, d_year#15] -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=5] -Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#31 IN dynamicpruning#32 -BroadcastExchange (79) -+- * ColumnarToRow (78) - +- CometFilter (77) - +- CometScan parquet spark_catalog.default.date_dim (76) +Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#31 +BroadcastExchange (76) ++- * ColumnarToRow (75) + +- CometFilter (74) + +- CometScan parquet spark_catalog.default.date_dim (73) -(76) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_year#34] +(73) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#32, d_year#33] 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#33, d_year#34] -Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2002)) AND isnotnull(d_date_sk#33)) +(74) CometFilter +Input [2]: [d_date_sk#32, d_year#33] +Condition : ((isnotnull(d_year#33) AND (d_year#33 = 2002)) AND isnotnull(d_date_sk#32)) -(78) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#33, d_year#34] +(75) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#32, d_year#33] -(79) BroadcastExchange -Input [2]: [d_date_sk#33, d_year#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] +(76) BroadcastExchange +Input [2]: [d_date_sk#32, d_year#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -Subquery:3 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#13 +Subquery:3 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#13 -Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#71 IN dynamicpruning#32 +Subquery:4 Hosting operator id = 53 Hosting Expression = ws_sold_date_sk#69 IN dynamicpruning#31 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 25010c5eb..c4811d292 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,106 +1,86 @@ -TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_email_address] - 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] - BroadcastHashJoin [customer_id,customer_id] - BroadcastHashJoin [customer_id,customer_id] - Filter [year_total] - 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 (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,sum,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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_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_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - 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,sum,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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 - CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_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 #8 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - 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] #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,sum,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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 - CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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 #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] #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,sum,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 [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,d_date_sk,d_year] - 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 - CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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 +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_email_address] + CometProject [customer_id,customer_first_name,customer_last_name,customer_email_address] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total,customer_id,year_total] + CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,customer_id,year_total] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total] + CometFilter [customer_id,year_total] + CometHashAggregate [customer_id,year_total,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)))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + 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 [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_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_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,customer_email_address,year_total] #5 + CometHashAggregate [customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,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)))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #6 + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + 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 [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_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 #8 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [customer_id,year_total] #10 + CometFilter [customer_id,year_total] + CometHashAggregate [customer_id,year_total,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)))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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 + CometBroadcastExchange [customer_id,year_total] #13 + CometHashAggregate [customer_id,year_total,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)))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,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 [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,d_date_sk,d_year] + 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,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + 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 [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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 04b5d41b5..fbe5feb3e 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 @@ -2,11 +2,11 @@ TakeOrderedAndProject (22) +- * Project (21) +- Window (20) - +- * Sort (19) - +- Exchange (18) - +- * HashAggregate (17) - +- Exchange (16) - +- * ColumnarToRow (15) + +- * ColumnarToRow (19) + +- CometSort (18) + +- CometColumnarExchange (17) + +- CometHashAggregate (16) + +- CometColumnarExchange (15) +- CometHashAggregate (14) +- CometProject (13) +- CometBroadcastHashJoin (12) @@ -92,39 +92,37 @@ Input [6]: [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, 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))] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(16) CometHashAggregate 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] -(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=2] +(17) CometColumnarExchange +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(18) CometSort +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] +Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15], [i_class#9 ASC NULLS FIRST] -(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 +(19) ColumnarToRow [codegen id : 1] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] (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] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] +Arguments: [sum(_w0#15) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#16], [i_class#9] -(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] +(21) Project [codegen id : 2] +Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, ((_w0#15 * 100) / _we0#16) AS revenueratio#17] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, _we0#16] (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] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] +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#17 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] ===== Subqueries ===== 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 a8b8a9cce..e2df89eec 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,38 +1,34 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (4) + WholeStageCodegen (2) Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] InputAdapter Window [_w0,i_class] - WholeStageCodegen (3) - Sort [i_class] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [i_class] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ws_ext_sales_price] - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometColumnarExchange [i_class] #1 + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ws_ext_sales_price] + CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + 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 cded32ec2..90e7b6ca1 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 @@ -1,88 +1,89 @@ == Physical Plan == -TakeOrderedAndProject (84) -+- * BroadcastHashJoin Inner BuildRight (83) - :- * Filter (66) - : +- * HashAggregate (65) - : +- Exchange (64) - : +- * HashAggregate (63) - : +- * Project (62) - : +- * BroadcastHashJoin Inner BuildRight (61) - : :- * Project (59) - : : +- * BroadcastHashJoin Inner BuildRight (58) - : : :- * BroadcastHashJoin LeftSemi BuildRight (51) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (50) - : : : +- * Project (49) - : : : +- * BroadcastHashJoin Inner BuildRight (48) - : : : :- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.item (4) - : : : +- BroadcastExchange (47) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) - : : : :- * HashAggregate (35) - : : : : +- Exchange (34) - : : : : +- * 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) - : : : +- * 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) - : : : +- CometFilter (53) - : : : +- CometScan parquet spark_catalog.default.item (52) - : : +- ReusedExchange (55) - : +- ReusedExchange (60) - +- BroadcastExchange (82) - +- * Filter (81) - +- * HashAggregate (80) - +- Exchange (79) - +- * HashAggregate (78) - +- * Project (77) - +- * BroadcastHashJoin Inner BuildRight (76) - :- * Project (74) - : +- * BroadcastHashJoin Inner BuildRight (73) - : :- * BroadcastHashJoin LeftSemi BuildRight (71) - : : :- * ColumnarToRow (69) - : : : +- CometFilter (68) - : : : +- CometScan parquet spark_catalog.default.store_sales (67) - : : +- ReusedExchange (70) - : +- ReusedExchange (72) - +- ReusedExchange (75) +* ColumnarToRow (85) ++- CometTakeOrderedAndProject (84) + +- CometBroadcastHashJoin (83) + :- CometFilter (64) + : +- CometHashAggregate (63) + : +- CometColumnarExchange (62) + : +- CometHashAggregate (61) + : +- CometProject (60) + : +- CometBroadcastHashJoin (59) + : :- CometProject (54) + : : +- CometBroadcastHashJoin (53) + : : :- CometBroadcastHashJoin (47) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (46) + : : : +- CometProject (45) + : : : +- CometBroadcastHashJoin (44) + : : : :- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.item (3) + : : : +- CometBroadcastExchange (43) + : : : +- CometBroadcastHashJoin (42) + : : : :- CometHashAggregate (32) + : : : : +- CometColumnarExchange (31) + : : : : +- CometHashAggregate (30) + : : : : +- CometProject (29) + : : : : +- CometBroadcastHashJoin (28) + : : : : :- CometProject (26) + : : : : : +- CometBroadcastHashJoin (25) + : : : : : :- CometFilter (6) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (5) + : : : : : +- CometBroadcastExchange (24) + : : : : : +- CometBroadcastHashJoin (23) + : : : : : :- CometFilter (8) + : : : : : : +- CometScan parquet spark_catalog.default.item (7) + : : : : : +- CometBroadcastExchange (22) + : : : : : +- CometProject (21) + : : : : : +- CometBroadcastHashJoin (20) + : : : : : :- CometProject (15) + : : : : : : +- CometBroadcastHashJoin (14) + : : : : : : :- CometFilter (10) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (9) + : : : : : : +- CometBroadcastExchange (13) + : : : : : : +- CometFilter (12) + : : : : : : +- CometScan parquet spark_catalog.default.item (11) + : : : : : +- CometBroadcastExchange (19) + : : : : : +- CometProject (18) + : : : : : +- CometFilter (17) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (16) + : : : : +- ReusedExchange (27) + : : : +- CometBroadcastExchange (41) + : : : +- CometProject (40) + : : : +- CometBroadcastHashJoin (39) + : : : :- CometProject (37) + : : : : +- CometBroadcastHashJoin (36) + : : : : :- CometFilter (34) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (33) + : : : : +- ReusedExchange (35) + : : : +- ReusedExchange (38) + : : +- CometBroadcastExchange (52) + : : +- CometBroadcastHashJoin (51) + : : :- CometFilter (49) + : : : +- CometScan parquet spark_catalog.default.item (48) + : : +- ReusedExchange (50) + : +- CometBroadcastExchange (58) + : +- CometProject (57) + : +- CometFilter (56) + : +- CometScan parquet spark_catalog.default.date_dim (55) + +- CometBroadcastExchange (82) + +- CometFilter (81) + +- CometHashAggregate (80) + +- CometColumnarExchange (79) + +- CometHashAggregate (78) + +- CometProject (77) + +- CometBroadcastHashJoin (76) + :- CometProject (71) + : +- CometBroadcastHashJoin (70) + : :- CometBroadcastHashJoin (68) + : : :- CometFilter (66) + : : : +- CometScan parquet spark_catalog.default.store_sales (65) + : : +- ReusedExchange (67) + : +- ReusedExchange (69) + +- CometBroadcastExchange (75) + +- CometProject (74) + +- CometFilter (73) + +- CometScan parquet spark_catalog.default.date_dim (72) (1) Scan parquet spark_catalog.default.store_sales @@ -97,24 +98,18 @@ 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 : 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 +(3) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) -(6) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(7) Scan parquet spark_catalog.default.store_sales +(5) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] @@ -122,22 +117,22 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(8) CometFilter +(6) CometFilter Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(9) Scan parquet spark_catalog.default.item +(7) 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 -(10) CometFilter +(8) 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)) -(11) Scan parquet spark_catalog.default.catalog_sales +(9) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] @@ -145,116 +140,111 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(12) CometFilter +(10) CometFilter Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) -(13) Scan parquet spark_catalog.default.item +(11) 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 -(14) CometFilter +(12) 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) -(15) CometBroadcastExchange +(13) 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] -(16) CometBroadcastHashJoin +(14) 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 -(17) CometProject +(15) 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 +(16) 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 -(19) CometFilter +(17) 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 +(18) CometProject Input [2]: [d_date_sk#24, d_year#25] Arguments: [d_date_sk#24], [d_date_sk#24] -(21) CometBroadcastExchange +(19) CometBroadcastExchange Input [1]: [d_date_sk#24] Arguments: [d_date_sk#24] -(22) CometBroadcastHashJoin +(20) 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 +(21) 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] -(24) CometBroadcastExchange +(22) CometBroadcastExchange Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] -(25) CometBroadcastHashJoin +(23) 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 -(26) CometBroadcastExchange +(24) CometBroadcastExchange Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(27) CometBroadcastHashJoin +(25) 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 -(28) CometProject +(26) 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] -(29) ReusedExchange [Reuses operator id: 21] +(27) ReusedExchange [Reuses operator id: 19] Output [1]: [d_date_sk#26] -(30) CometBroadcastHashJoin +(28) 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 -(31) CometProject +(29) 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] -(32) CometHashAggregate +(30) CometHashAggregate Input [3]: [brand_id#27, class_id#28, category_id#29] Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] -(33) ColumnarToRow [codegen id : 1] -Input [3]: [brand_id#27, class_id#28, category_id#29] - -(34) Exchange +(31) CometColumnarExchange 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] +Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(35) HashAggregate [codegen id : 3] +(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#27, class_id#28, category_id#29] -(36) Scan parquet spark_catalog.default.web_sales +(33) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] @@ -262,482 +252,492 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(37) CometFilter +(34) CometFilter Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] Condition : isnotnull(ws_item_sk#30) -(38) ReusedExchange [Reuses operator id: 15] +(35) ReusedExchange [Reuses operator id: 13] Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -(39) CometBroadcastHashJoin +(36) 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) CometProject +(37) 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) ReusedExchange [Reuses operator id: 21] +(38) ReusedExchange [Reuses operator id: 19] Output [1]: [d_date_sk#37] -(42) CometBroadcastHashJoin +(39) 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) CometProject +(40) 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) ColumnarToRow [codegen id : 2] -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(45) BroadcastExchange +(41) CometBroadcastExchange 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] +Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] -(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 +(42) CometBroadcastHashJoin +Left output [3]: [brand_id#27, class_id#28, category_id#29] +Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [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)], [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)], LeftSemi, BuildRight -(47) BroadcastExchange +(43) CometBroadcastExchange 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] +Arguments: [brand_id#27, class_id#28, category_id#29] -(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#27, class_id#28, category_id#29] -Join type: Inner -Join condition: None +(44) CometBroadcastHashJoin +Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Right output [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight -(49) Project [codegen id : 4] -Output [1]: [i_item_sk#6 AS ss_item_sk#38] +(45) CometProject 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] +Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] -(50) BroadcastExchange +(46) CometBroadcastExchange Input [1]: [ss_item_sk#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: [ss_item_sk#38] -(51) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#38] -Join type: LeftSemi -Join condition: None +(47) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Right output [1]: [ss_item_sk#38] +Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight -(52) Scan parquet spark_catalog.default.item +(48) Scan parquet spark_catalog.default.item 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 +(49) CometFilter 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 : 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] +(50) ReusedExchange [Reuses operator id: 46] Output [1]: [ss_item_sk#38] -(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 +(51) CometBroadcastHashJoin +Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Right output [1]: [ss_item_sk#38] +Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight -(57) BroadcastExchange +(52) CometBroadcastExchange 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] +Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -(58) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#39] -Join type: Inner -Join condition: None +(53) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight -(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] +(54) CometProject 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] +Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -(60) ReusedExchange [Reuses operator id: 106] -Output [1]: [d_date_sk#43] +(55) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#43, d_week_seq#44] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] +ReadSchema: struct -(61) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#43] -Join type: Inner -Join condition: None +(56) CometFilter +Input [2]: [d_date_sk#43, d_week_seq#44] +Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = ReusedSubquery Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) -(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] +(57) CometProject +Input [2]: [d_date_sk#43, d_week_seq#44] +Arguments: [d_date_sk#43], [d_date_sk#43] + +(58) CometBroadcastExchange +Input [1]: [d_date_sk#43] +Arguments: [d_date_sk#43] + +(59) CometBroadcastHashJoin +Left 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] +Right output [1]: [d_date_sk#43] +Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight + +(60) CometProject 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] +Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -(63) HashAggregate [codegen id : 11] +(61) CometHashAggregate 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#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 +(62) CometColumnarExchange 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] +Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(65) HashAggregate [codegen id : 24] +(63) CometHashAggregate 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))#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 : 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)))) +(64) CometFilter +Input [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, 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)))) -(67) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60] +(65) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#60), dynamicpruningexpression(ss_sold_date_sk#60 IN dynamicpruning#61)] +PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(68) CometFilter -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 : 22] -Input [4]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60] +(66) CometFilter +Input [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] +Condition : isnotnull(ss_item_sk#55) -(70) ReusedExchange [Reuses operator id: 50] +(67) ReusedExchange [Reuses operator id: 46] Output [1]: [ss_item_sk#38] -(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#62, i_brand_id#63, i_class_id#64, i_category_id#65] - -(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 : 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: 120] -Output [1]: [d_date_sk#66] - -(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 : 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 : 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#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 : 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 : 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#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 : 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#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] +(68) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] +Right output [1]: [ss_item_sk#38] +Arguments: [ss_item_sk#55], [ss_item_sk#38], LeftSemi, BuildRight + +(69) ReusedExchange [Reuses operator id: 52] +Output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] + +(70) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] +Right output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] +Arguments: [ss_item_sk#55], [i_item_sk#60], Inner, BuildRight + +(71) CometProject +Input [8]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] +Arguments: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#61, i_class_id#62, i_category_id#63], [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#61, i_class_id#62, i_category_id#63] + +(72) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#64, d_week_seq#65] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] +ReadSchema: struct + +(73) CometFilter +Input [2]: [d_date_sk#64, d_week_seq#65] +Condition : ((isnotnull(d_week_seq#65) AND (d_week_seq#65 = ReusedSubquery Subquery scalar-subquery#66, [id=#67])) AND isnotnull(d_date_sk#64)) + +(74) CometProject +Input [2]: [d_date_sk#64, d_week_seq#65] +Arguments: [d_date_sk#64], [d_date_sk#64] + +(75) CometBroadcastExchange +Input [1]: [d_date_sk#64] +Arguments: [d_date_sk#64] + +(76) CometBroadcastHashJoin +Left output [6]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#61, i_class_id#62, i_category_id#63] +Right output [1]: [d_date_sk#64] +Arguments: [ss_sold_date_sk#58], [d_date_sk#64], Inner, BuildRight + +(77) CometProject +Input [7]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#61, i_class_id#62, i_category_id#63, d_date_sk#64] +Arguments: [ss_quantity#56, ss_list_price#57, i_brand_id#61, i_class_id#62, i_category_id#63], [ss_quantity#56, ss_list_price#57, i_brand_id#61, i_class_id#62, i_category_id#63] + +(78) CometHashAggregate +Input [5]: [ss_quantity#56, ss_list_price#57, i_brand_id#61, i_class_id#62, i_category_id#63] +Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] +Functions [2]: [partial_sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), partial_count(1)] + +(79) CometColumnarExchange +Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] +Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(80) CometHashAggregate +Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] +Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] +Functions [2]: [sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), count(1)] + +(81) CometFilter +Input [6]: [channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] +Condition : (isnotnull(sales#72) AND (cast(sales#72 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) + +(82) CometBroadcastExchange +Input [6]: [channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] +Arguments: [channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] + +(83) CometBroadcastHashJoin +Left output [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] +Right output [6]: [channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] +Arguments: [i_brand_id#40, i_class_id#41, i_category_id#42], [i_brand_id#61, i_class_id#62, i_category_id#63], Inner, BuildRight + +(84) CometTakeOrderedAndProject +Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#50,i_brand_id#40,i_class_id#41,i_category_id#42,sales#51,number_sales#52,channel#71,i_brand_id#61,i_class_id#62,i_category_id#63,sales#72,number_sales#73]), [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73], 100, [i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] + +(85) ColumnarToRow [codegen id : 1] +Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] ===== Subqueries ===== -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#78, ss_list_price#79, ss_sold_date_sk#80] +Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#53, [id=#54] +* ColumnarToRow (102) ++- CometHashAggregate (101) + +- CometColumnarExchange (100) + +- CometHashAggregate (99) + +- CometUnion (98) + :- CometProject (89) + : +- CometBroadcastHashJoin (88) + : :- CometScan parquet spark_catalog.default.store_sales (86) + : +- ReusedExchange (87) + :- CometProject (93) + : +- CometBroadcastHashJoin (92) + : :- CometScan parquet spark_catalog.default.catalog_sales (90) + : +- ReusedExchange (91) + +- CometProject (97) + +- CometBroadcastHashJoin (96) + :- CometScan parquet spark_catalog.default.web_sales (94) + +- ReusedExchange (95) + + +(86) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_quantity#74, ss_list_price#75, ss_sold_date_sk#76] 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#76), dynamicpruningexpression(ss_sold_date_sk#76 IN dynamicpruning#77)] ReadSchema: struct -(86) ReusedExchange [Reuses operator id: 21] -Output [1]: [d_date_sk#82] +(87) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#78] -(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 +(88) CometBroadcastHashJoin +Left output [3]: [ss_quantity#74, ss_list_price#75, ss_sold_date_sk#76] +Right output [1]: [d_date_sk#78] +Arguments: [ss_sold_date_sk#76], [d_date_sk#78], Inner, BuildRight -(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] +(89) CometProject +Input [4]: [ss_quantity#74, ss_list_price#75, ss_sold_date_sk#76, d_date_sk#78] +Arguments: [quantity#79, list_price#80], [ss_quantity#74 AS quantity#79, ss_list_price#75 AS list_price#80] -(89) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87] +(90) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83] 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#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#84)] ReadSchema: struct -(90) ReusedExchange [Reuses operator id: 21] -Output [1]: [d_date_sk#89] +(91) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#85] -(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 +(92) CometBroadcastHashJoin +Left output [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83] +Right output [1]: [d_date_sk#85] +Arguments: [cs_sold_date_sk#83], [d_date_sk#85], Inner, BuildRight -(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] +(93) CometProject +Input [4]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83, d_date_sk#85] +Arguments: [quantity#86, list_price#87], [cs_quantity#81 AS quantity#86, cs_list_price#82 AS list_price#87] -(93) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] +(94) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90] 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#90), dynamicpruningexpression(ws_sold_date_sk#90 IN dynamicpruning#91)] ReadSchema: struct -(94) ReusedExchange [Reuses operator id: 21] -Output [1]: [d_date_sk#96] +(95) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#92] -(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 +(96) CometBroadcastHashJoin +Left output [3]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90] +Right output [1]: [d_date_sk#92] +Arguments: [ws_sold_date_sk#90], [d_date_sk#92], Inner, BuildRight -(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] +(97) CometProject +Input [4]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90, d_date_sk#92] +Arguments: [quantity#93, list_price#94], [ws_quantity#88 AS quantity#93, ws_list_price#89 AS list_price#94] -(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] +(98) CometUnion +Child 0 Input [2]: [quantity#79, list_price#80] +Child 1 Input [2]: [quantity#86, list_price#87] +Child 2 Input [2]: [quantity#93, list_price#94] -(98) CometHashAggregate -Input [2]: [quantity#83, list_price#84] +(99) CometHashAggregate +Input [2]: [quantity#79, list_price#80] Keys: [] -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] +Functions [1]: [partial_avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] -(100) Exchange -Input [2]: [sum#99, count#100] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] +(100) CometColumnarExchange +Input [2]: [sum#95, count#96] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(101) HashAggregate [codegen id : 2] -Input [2]: [sum#99, count#100] +(101) CometHashAggregate +Input [2]: [sum#95, count#96] 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))#101] -Results [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))#101 AS average_sales#102] +Functions [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] -Subquery:2 Hosting operator id = 85 Hosting Expression = ss_sold_date_sk#80 IN dynamicpruning#12 +(102) ColumnarToRow [codegen id : 1] +Input [1]: [average_sales#97] -Subquery:3 Hosting operator id = 89 Hosting Expression = cs_sold_date_sk#87 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 86 Hosting Expression = ss_sold_date_sk#76 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 93 Hosting Expression = ws_sold_date_sk#94 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#12 + +Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#90 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (106) -+- * ColumnarToRow (105) - +- CometProject (104) - +- CometFilter (103) - +- CometScan parquet spark_catalog.default.date_dim (102) +BroadcastExchange (107) ++- * ColumnarToRow (106) + +- CometProject (105) + +- CometFilter (104) + +- CometScan parquet spark_catalog.default.date_dim (103) -(102) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_week_seq#103] +(103) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#43, d_week_seq#44] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(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)) +(104) CometFilter +Input [2]: [d_date_sk#43, d_week_seq#44] +Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) -(104) CometProject -Input [2]: [d_date_sk#43, d_week_seq#103] +(105) CometProject +Input [2]: [d_date_sk#43, d_week_seq#44] Arguments: [d_date_sk#43], [d_date_sk#43] -(105) ColumnarToRow [codegen id : 1] +(106) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] -(106) BroadcastExchange +(107) BroadcastExchange Input [1]: [d_date_sk#43] -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=5] -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) +Subquery:6 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] +* ColumnarToRow (111) ++- CometProject (110) + +- CometFilter (109) + +- CometScan parquet spark_catalog.default.date_dim (108) -(107) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#106, d_year#107, d_moy#108, d_dom#109] +(108) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] 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 -(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)) +(109) CometFilter +Input [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] +Condition : (((((isnotnull(d_year#99) AND isnotnull(d_moy#100)) AND isnotnull(d_dom#101)) AND (d_year#99 = 1999)) AND (d_moy#100 = 12)) AND (d_dom#101 = 16)) -(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] +(110) CometProject +Input [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] +Arguments: [d_week_seq#98], [d_week_seq#98] -(110) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#106] +(111) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#98] -Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (115) -+- * ColumnarToRow (114) - +- CometProject (113) - +- CometFilter (112) - +- CometScan parquet spark_catalog.default.date_dim (111) +Subquery:7 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 +BroadcastExchange (116) ++- * ColumnarToRow (115) + +- CometProject (114) + +- CometFilter (113) + +- CometScan parquet spark_catalog.default.date_dim (112) -(111) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#110] +(112) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_year#102] 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 -(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)) +(113) CometFilter +Input [2]: [d_date_sk#26, d_year#102] +Condition : (((isnotnull(d_year#102) AND (d_year#102 >= 1998)) AND (d_year#102 <= 2000)) AND isnotnull(d_date_sk#26)) -(113) CometProject -Input [2]: [d_date_sk#26, d_year#110] +(114) CometProject +Input [2]: [d_date_sk#26, d_year#102] Arguments: [d_date_sk#26], [d_date_sk#26] -(114) ColumnarToRow [codegen id : 1] +(115) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] -(115) BroadcastExchange +(116) BroadcastExchange Input [1]: [d_date_sk#26] -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=6] + +Subquery:8 Hosting operator id = 9 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 = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 +Subquery:10 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:10 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#55, [id=#56] +Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#53, [id=#54] -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) +Subquery:12 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 +BroadcastExchange (121) ++- * ColumnarToRow (120) + +- CometProject (119) + +- CometFilter (118) + +- CometScan parquet spark_catalog.default.date_dim (117) -(116) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#66, d_week_seq#111] +(117) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#64, d_week_seq#65] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(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)) +(118) CometFilter +Input [2]: [d_date_sk#64, d_week_seq#65] +Condition : ((isnotnull(d_week_seq#65) AND (d_week_seq#65 = Subquery scalar-subquery#66, [id=#67])) AND isnotnull(d_date_sk#64)) -(118) CometProject -Input [2]: [d_date_sk#66, d_week_seq#111] -Arguments: [d_date_sk#66], [d_date_sk#66] +(119) CometProject +Input [2]: [d_date_sk#64, d_week_seq#65] +Arguments: [d_date_sk#64], [d_date_sk#64] -(119) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#66] +(120) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#64] -(120) BroadcastExchange -Input [1]: [d_date_sk#66] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] +(121) BroadcastExchange +Input [1]: [d_date_sk#64] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -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) +Subquery:13 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#66, [id=#67] +* ColumnarToRow (125) ++- CometProject (124) + +- CometFilter (123) + +- CometScan parquet spark_catalog.default.date_dim (122) -(121) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#114, d_year#115, d_moy#116, d_dom#117] +(122) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_week_seq#103, d_year#104, d_moy#105, d_dom#106] 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 -(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)) +(123) CometFilter +Input [4]: [d_week_seq#103, d_year#104, d_moy#105, d_dom#106] +Condition : (((((isnotnull(d_year#104) AND isnotnull(d_moy#105)) AND isnotnull(d_dom#106)) AND (d_year#104 = 1998)) AND (d_moy#105 = 12)) AND (d_dom#106 = 16)) + +(124) CometProject +Input [4]: [d_week_seq#103, d_year#104, d_moy#105, d_dom#106] +Arguments: [d_week_seq#103], [d_week_seq#103] -(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] +(125) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#103] -(124) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#114] +Subquery:14 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] 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 aa49638c7..a51d1d007 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,15 +1,15 @@ -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 (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 (2) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - InputAdapter - Exchange #13 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + CometBroadcastHashJoin [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Subquery #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] + CometColumnarExchange #14 CometHashAggregate [sum,count,quantity,list_price] CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] @@ -27,152 +27,127 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ 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 (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] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (4) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #1 + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ss_quantity,ss_list_price] + CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) 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 #4 - 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 (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,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 - CometBroadcastHashJoin [i_item_sk,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 [i_brand_id,i_class_id,i_category_id] #8 - CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [cs_item_sk,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - 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 [d_date_sk] #10 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #10 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (2) + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometBroadcastExchange [ss_item_sk] #3 + CometProject [i_item_sk] [ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,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 [brand_id,class_id,category_id] #4 + CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + CometHashAggregate [brand_id,class_id,category_id] + CometColumnarExchange [brand_id,class_id,category_id] #5 + 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,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 + CometBroadcastHashJoin [i_item_sk,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 [i_brand_id,i_class_id,i_category_id] #8 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + 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 [d_date_sk] #10 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #10 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #11 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + ReusedExchange [d_date_sk] #10 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] + 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] + ReusedExchange [ss_item_sk] #3 + CometBroadcastExchange [d_date_sk] #13 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #2 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometBroadcastExchange [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] #15 + CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + ReusedSubquery [average_sales] #4 + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #16 + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ss_quantity,ss_list_price] + CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_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 #17 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #6 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ws_item_sk,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - ReusedExchange [d_date_sk] #10 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - BroadcastHashJoin [i_item_sk,ss_item_sk] - 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 - ReusedExchange [ss_item_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - 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] #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] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_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 #16 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - ReusedExchange [ss_item_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - InputAdapter - ReusedExchange [d_date_sk] #16 + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + ReusedExchange [ss_item_sk] #3 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + CometBroadcastExchange [d_date_sk] #18 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #6 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] 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 611980392..0a4f72baa 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 @@ -1,129 +1,126 @@ == Physical Plan == -TakeOrderedAndProject (125) -+- * HashAggregate (124) - +- Exchange (123) - +- * HashAggregate (122) - +- Union (121) - :- * HashAggregate (100) - : +- Exchange (99) - : +- * HashAggregate (98) - : +- Union (97) - : :- * Filter (66) - : : +- * HashAggregate (65) - : : +- Exchange (64) - : : +- * HashAggregate (63) - : : +- * Project (62) - : : +- * BroadcastHashJoin Inner BuildRight (61) - : : :- * Project (59) - : : : +- * BroadcastHashJoin Inner BuildRight (58) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (51) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- BroadcastExchange (50) - : : : : +- * Project (49) - : : : : +- * BroadcastHashJoin Inner BuildRight (48) - : : : : :- * ColumnarToRow (6) - : : : : : +- CometFilter (5) - : : : : : +- CometScan parquet spark_catalog.default.item (4) - : : : : +- BroadcastExchange (47) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) - : : : : :- * HashAggregate (35) - : : : : : +- Exchange (34) - : : : : : +- * 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) - : : : : +- * 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) - : : : : +- CometFilter (53) - : : : : +- CometScan parquet spark_catalog.default.item (52) - : : : +- ReusedExchange (55) - : : +- ReusedExchange (60) - : :- * Filter (81) - : : +- * HashAggregate (80) - : : +- Exchange (79) - : : +- * HashAggregate (78) - : : +- * Project (77) - : : +- * BroadcastHashJoin Inner BuildRight (76) - : : :- * Project (74) - : : : +- * BroadcastHashJoin Inner BuildRight (73) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (71) - : : : : :- * ColumnarToRow (69) - : : : : : +- CometFilter (68) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (67) - : : : : +- ReusedExchange (70) - : : : +- ReusedExchange (72) - : : +- ReusedExchange (75) - : +- * Filter (96) - : +- * HashAggregate (95) - : +- Exchange (94) - : +- * HashAggregate (93) - : +- * Project (92) - : +- * BroadcastHashJoin Inner BuildRight (91) - : :- * Project (89) - : : +- * BroadcastHashJoin Inner BuildRight (88) - : : :- * BroadcastHashJoin LeftSemi BuildRight (86) - : : : :- * ColumnarToRow (84) - : : : : +- CometFilter (83) - : : : : +- CometScan parquet spark_catalog.default.web_sales (82) - : : : +- ReusedExchange (85) - : : +- ReusedExchange (87) - : +- ReusedExchange (90) - :- * HashAggregate (105) - : +- Exchange (104) - : +- * HashAggregate (103) - : +- * HashAggregate (102) - : +- ReusedExchange (101) - :- * HashAggregate (110) - : +- Exchange (109) - : +- * HashAggregate (108) - : +- * HashAggregate (107) - : +- ReusedExchange (106) - :- * HashAggregate (115) - : +- Exchange (114) - : +- * HashAggregate (113) - : +- * HashAggregate (112) - : +- ReusedExchange (111) - +- * HashAggregate (120) - +- Exchange (119) - +- * HashAggregate (118) - +- * HashAggregate (117) - +- ReusedExchange (116) +* ColumnarToRow (122) ++- CometTakeOrderedAndProject (121) + +- CometHashAggregate (120) + +- CometColumnarExchange (119) + +- CometHashAggregate (118) + +- CometUnion (117) + :- CometHashAggregate (96) + : +- CometColumnarExchange (95) + : +- CometHashAggregate (94) + : +- CometUnion (93) + : :- CometFilter (64) + : : +- CometHashAggregate (63) + : : +- CometColumnarExchange (62) + : : +- CometHashAggregate (61) + : : +- CometProject (60) + : : +- CometBroadcastHashJoin (59) + : : :- CometProject (54) + : : : +- CometBroadcastHashJoin (53) + : : : :- CometBroadcastHashJoin (47) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometBroadcastExchange (46) + : : : : +- CometProject (45) + : : : : +- CometBroadcastHashJoin (44) + : : : : :- CometFilter (4) + : : : : : +- CometScan parquet spark_catalog.default.item (3) + : : : : +- CometBroadcastExchange (43) + : : : : +- CometBroadcastHashJoin (42) + : : : : :- CometHashAggregate (32) + : : : : : +- CometColumnarExchange (31) + : : : : : +- CometHashAggregate (30) + : : : : : +- CometProject (29) + : : : : : +- CometBroadcastHashJoin (28) + : : : : : :- CometProject (26) + : : : : : : +- CometBroadcastHashJoin (25) + : : : : : : :- CometFilter (6) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (5) + : : : : : : +- CometBroadcastExchange (24) + : : : : : : +- CometBroadcastHashJoin (23) + : : : : : : :- CometFilter (8) + : : : : : : : +- CometScan parquet spark_catalog.default.item (7) + : : : : : : +- CometBroadcastExchange (22) + : : : : : : +- CometProject (21) + : : : : : : +- CometBroadcastHashJoin (20) + : : : : : : :- CometProject (15) + : : : : : : : +- CometBroadcastHashJoin (14) + : : : : : : : :- CometFilter (10) + : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (9) + : : : : : : : +- CometBroadcastExchange (13) + : : : : : : : +- CometFilter (12) + : : : : : : : +- CometScan parquet spark_catalog.default.item (11) + : : : : : : +- CometBroadcastExchange (19) + : : : : : : +- CometProject (18) + : : : : : : +- CometFilter (17) + : : : : : : +- CometScan parquet spark_catalog.default.date_dim (16) + : : : : : +- ReusedExchange (27) + : : : : +- CometBroadcastExchange (41) + : : : : +- CometProject (40) + : : : : +- CometBroadcastHashJoin (39) + : : : : :- CometProject (37) + : : : : : +- CometBroadcastHashJoin (36) + : : : : : :- CometFilter (34) + : : : : : : +- CometScan parquet spark_catalog.default.web_sales (33) + : : : : : +- ReusedExchange (35) + : : : : +- ReusedExchange (38) + : : : +- CometBroadcastExchange (52) + : : : +- CometBroadcastHashJoin (51) + : : : :- CometFilter (49) + : : : : +- CometScan parquet spark_catalog.default.item (48) + : : : +- ReusedExchange (50) + : : +- CometBroadcastExchange (58) + : : +- CometProject (57) + : : +- CometFilter (56) + : : +- CometScan parquet spark_catalog.default.date_dim (55) + : :- CometFilter (78) + : : +- CometHashAggregate (77) + : : +- CometColumnarExchange (76) + : : +- CometHashAggregate (75) + : : +- CometProject (74) + : : +- CometBroadcastHashJoin (73) + : : :- CometProject (71) + : : : +- CometBroadcastHashJoin (70) + : : : :- CometBroadcastHashJoin (68) + : : : : :- CometFilter (66) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (65) + : : : : +- ReusedExchange (67) + : : : +- ReusedExchange (69) + : : +- ReusedExchange (72) + : +- CometFilter (92) + : +- CometHashAggregate (91) + : +- CometColumnarExchange (90) + : +- CometHashAggregate (89) + : +- CometProject (88) + : +- CometBroadcastHashJoin (87) + : :- CometProject (85) + : : +- CometBroadcastHashJoin (84) + : : :- CometBroadcastHashJoin (82) + : : : :- CometFilter (80) + : : : : +- CometScan parquet spark_catalog.default.web_sales (79) + : : : +- ReusedExchange (81) + : : +- ReusedExchange (83) + : +- ReusedExchange (86) + :- CometHashAggregate (101) + : +- CometColumnarExchange (100) + : +- CometHashAggregate (99) + : +- CometHashAggregate (98) + : +- ReusedExchange (97) + :- CometHashAggregate (106) + : +- CometColumnarExchange (105) + : +- CometHashAggregate (104) + : +- CometHashAggregate (103) + : +- ReusedExchange (102) + :- CometHashAggregate (111) + : +- CometColumnarExchange (110) + : +- CometHashAggregate (109) + : +- CometHashAggregate (108) + : +- ReusedExchange (107) + +- CometHashAggregate (116) + +- CometColumnarExchange (115) + +- CometHashAggregate (114) + +- CometHashAggregate (113) + +- ReusedExchange (112) (1) Scan parquet spark_catalog.default.store_sales @@ -138,24 +135,18 @@ 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 : 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 +(3) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) -(6) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(7) Scan parquet spark_catalog.default.store_sales +(5) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] @@ -163,22 +154,22 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(8) CometFilter +(6) CometFilter Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(9) Scan parquet spark_catalog.default.item +(7) 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 -(10) CometFilter +(8) 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)) -(11) Scan parquet spark_catalog.default.catalog_sales +(9) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] @@ -186,116 +177,111 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(12) CometFilter +(10) CometFilter Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) -(13) Scan parquet spark_catalog.default.item +(11) 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 -(14) CometFilter +(12) 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) -(15) CometBroadcastExchange +(13) 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] -(16) CometBroadcastHashJoin +(14) 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 -(17) CometProject +(15) 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 +(16) 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 -(19) CometFilter +(17) 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 +(18) CometProject Input [2]: [d_date_sk#24, d_year#25] Arguments: [d_date_sk#24], [d_date_sk#24] -(21) CometBroadcastExchange +(19) CometBroadcastExchange Input [1]: [d_date_sk#24] Arguments: [d_date_sk#24] -(22) CometBroadcastHashJoin +(20) 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 +(21) 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] -(24) CometBroadcastExchange +(22) CometBroadcastExchange Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] -(25) CometBroadcastHashJoin +(23) 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 -(26) CometBroadcastExchange +(24) CometBroadcastExchange Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(27) CometBroadcastHashJoin +(25) 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 -(28) CometProject +(26) 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] -(29) ReusedExchange [Reuses operator id: 21] +(27) ReusedExchange [Reuses operator id: 19] Output [1]: [d_date_sk#26] -(30) CometBroadcastHashJoin +(28) 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 -(31) CometProject +(29) 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] -(32) CometHashAggregate +(30) CometHashAggregate Input [3]: [brand_id#27, class_id#28, category_id#29] Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] -(33) ColumnarToRow [codegen id : 1] +(31) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(34) Exchange -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 : 3] +(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#27, class_id#28, category_id#29] -(36) Scan parquet spark_catalog.default.web_sales +(33) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] @@ -303,669 +289,626 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(37) CometFilter +(34) CometFilter Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] Condition : isnotnull(ws_item_sk#30) -(38) ReusedExchange [Reuses operator id: 15] +(35) ReusedExchange [Reuses operator id: 13] Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -(39) CometBroadcastHashJoin +(36) 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) CometProject +(37) 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) ReusedExchange [Reuses operator id: 21] +(38) ReusedExchange [Reuses operator id: 19] Output [1]: [d_date_sk#37] -(42) CometBroadcastHashJoin +(39) 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) CometProject +(40) 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) ColumnarToRow [codegen id : 2] -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(45) BroadcastExchange +(41) CometBroadcastExchange 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] +Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] -(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 +(42) CometBroadcastHashJoin +Left output [3]: [brand_id#27, class_id#28, category_id#29] +Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [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)], [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)], LeftSemi, BuildRight -(47) BroadcastExchange +(43) CometBroadcastExchange 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] +Arguments: [brand_id#27, class_id#28, category_id#29] -(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#27, class_id#28, category_id#29] -Join type: Inner -Join condition: None +(44) CometBroadcastHashJoin +Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Right output [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight -(49) Project [codegen id : 4] -Output [1]: [i_item_sk#6 AS ss_item_sk#38] +(45) CometProject 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] +Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] -(50) BroadcastExchange +(46) CometBroadcastExchange Input [1]: [ss_item_sk#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: [ss_item_sk#38] -(51) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#38] -Join type: LeftSemi -Join condition: None +(47) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Right output [1]: [ss_item_sk#38] +Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight -(52) Scan parquet spark_catalog.default.item +(48) Scan parquet spark_catalog.default.item 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 +(49) CometFilter 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 : 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] +(50) ReusedExchange [Reuses operator id: 46] Output [1]: [ss_item_sk#38] -(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 +(51) CometBroadcastHashJoin +Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Right output [1]: [ss_item_sk#38] +Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight -(57) BroadcastExchange +(52) CometBroadcastExchange 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] +Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -(58) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#39] -Join type: Inner -Join condition: None +(53) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight -(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] +(54) CometProject 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] +Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -(60) ReusedExchange [Reuses operator id: 155] -Output [1]: [d_date_sk#43] +(55) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#43, d_year#44, d_moy#45] +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 + +(56) CometFilter +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) + +(57) CometProject +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Arguments: [d_date_sk#43], [d_date_sk#43] + +(58) CometBroadcastExchange +Input [1]: [d_date_sk#43] +Arguments: [d_date_sk#43] -(61) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#43] -Join type: Inner -Join condition: None +(59) CometBroadcastHashJoin +Left 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] +Right output [1]: [d_date_sk#43] +Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight -(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] +(60) CometProject 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] +Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -(63) HashAggregate [codegen id : 11] +(61) CometHashAggregate 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#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#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] +(62) CometColumnarExchange +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] +Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(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] +(63) CometHashAggregate +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] 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))#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 : 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)))) +(64) CometFilter +Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] +Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) -(67) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] +(65) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_sold_date_sk#60 IN dynamicpruning#61)] +PartitionFilters: [isnotnull(cs_sold_date_sk#57), dynamicpruningexpression(cs_sold_date_sk#57 IN dynamicpruning#58)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(68) CometFilter -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 : 23] -Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] +(66) CometFilter +Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] +Condition : isnotnull(cs_item_sk#54) -(70) ReusedExchange [Reuses operator id: 50] +(67) ReusedExchange [Reuses operator id: 46] Output [1]: [ss_item_sk#38] -(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#62, i_brand_id#63, i_class_id#64, i_category_id#65] - -(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 : 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: 155] -Output [1]: [d_date_sk#66] - -(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 : 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 : 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#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 : 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 : 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#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] +(68) CometBroadcastHashJoin +Left output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] +Right output [1]: [ss_item_sk#38] +Arguments: [cs_item_sk#54], [ss_item_sk#38], LeftSemi, BuildRight + +(69) ReusedExchange [Reuses operator id: 52] +Output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] + +(70) CometBroadcastHashJoin +Left output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] +Right output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] +Arguments: [cs_item_sk#54], [i_item_sk#59], Inner, BuildRight + +(71) CometProject +Input [8]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] +Arguments: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62], [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62] + +(72) ReusedExchange [Reuses operator id: 58] +Output [1]: [d_date_sk#63] + +(73) CometBroadcastHashJoin +Left output [6]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62] +Right output [1]: [d_date_sk#63] +Arguments: [cs_sold_date_sk#57], [d_date_sk#63], Inner, BuildRight + +(74) CometProject +Input [7]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62, d_date_sk#63] +Arguments: [cs_quantity#55, cs_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62], [cs_quantity#55, cs_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] + +(75) CometHashAggregate +Input [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] +Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] +Functions [2]: [partial_sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), partial_count(1)] + +(76) CometColumnarExchange +Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#64, isEmpty#65, count#66] +Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(77) CometHashAggregate +Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#64, isEmpty#65, count#66] +Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] +Functions [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), count(1)] + +(78) CometFilter +Input [6]: [channel#67, i_brand_id#60, i_class_id#61, i_category_id#62, sales#68, number_sales#69] +Condition : (isnotnull(sales#68) AND (cast(sales#68 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) + +(79) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#70, ws_quantity#71, ws_list_price#72, ws_sold_date_sk#73] 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#73), dynamicpruningexpression(ws_sold_date_sk#73 IN dynamicpruning#74)] 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) - -(84) ColumnarToRow [codegen id : 35] -Input [4]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] +(80) CometFilter +Input [4]: [ws_item_sk#70, ws_quantity#71, ws_list_price#72, ws_sold_date_sk#73] +Condition : isnotnull(ws_item_sk#70) -(85) ReusedExchange [Reuses operator id: 50] +(81) ReusedExchange [Reuses operator id: 46] Output [1]: [ss_item_sk#38] -(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#83, i_brand_id#84, i_class_id#85, i_category_id#86] - -(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 : 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: 155] -Output [1]: [d_date_sk#87] - -(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 : 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 : 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#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 : 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 : 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 : 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#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 : 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#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#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 : 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#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#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 : 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#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#52, sum#135, isEmpty#136, sum#137] -Arguments: hashpartitioning(channel#52, 5), ENSURE_REQUIREMENTS, [plan_id=12] - -(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#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum#102, isEmpty#103, sum#104] - -(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 : 193] -Input [2]: [sum_sales#107, number_sales#108] +(82) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#70, ws_quantity#71, ws_list_price#72, ws_sold_date_sk#73] +Right output [1]: [ss_item_sk#38] +Arguments: [ws_item_sk#70], [ss_item_sk#38], LeftSemi, BuildRight + +(83) ReusedExchange [Reuses operator id: 52] +Output [4]: [i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] + +(84) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#70, ws_quantity#71, ws_list_price#72, ws_sold_date_sk#73] +Right output [4]: [i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] +Arguments: [ws_item_sk#70], [i_item_sk#75], Inner, BuildRight + +(85) CometProject +Input [8]: [ws_item_sk#70, ws_quantity#71, ws_list_price#72, ws_sold_date_sk#73, i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] +Arguments: [ws_quantity#71, ws_list_price#72, ws_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78], [ws_quantity#71, ws_list_price#72, ws_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78] + +(86) ReusedExchange [Reuses operator id: 58] +Output [1]: [d_date_sk#79] + +(87) CometBroadcastHashJoin +Left output [6]: [ws_quantity#71, ws_list_price#72, ws_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78] +Right output [1]: [d_date_sk#79] +Arguments: [ws_sold_date_sk#73], [d_date_sk#79], Inner, BuildRight + +(88) CometProject +Input [7]: [ws_quantity#71, ws_list_price#72, ws_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78, d_date_sk#79] +Arguments: [ws_quantity#71, ws_list_price#72, i_brand_id#76, i_class_id#77, i_category_id#78], [ws_quantity#71, ws_list_price#72, i_brand_id#76, i_class_id#77, i_category_id#78] + +(89) CometHashAggregate +Input [5]: [ws_quantity#71, ws_list_price#72, i_brand_id#76, i_class_id#77, i_category_id#78] +Keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] +Functions [2]: [partial_sum((cast(ws_quantity#71 as decimal(10,0)) * ws_list_price#72)), partial_count(1)] + +(90) CometColumnarExchange +Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#80, isEmpty#81, count#82] +Arguments: hashpartitioning(i_brand_id#76, i_class_id#77, i_category_id#78, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(91) CometHashAggregate +Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#80, isEmpty#81, count#82] +Keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] +Functions [2]: [sum((cast(ws_quantity#71 as decimal(10,0)) * ws_list_price#72)), count(1)] + +(92) CometFilter +Input [6]: [channel#83, i_brand_id#76, i_class_id#77, i_category_id#78, sales#84, number_sales#85] +Condition : (isnotnull(sales#84) AND (cast(sales#84 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) + +(93) CometUnion +Child 0 Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] +Child 1 Input [6]: [channel#67, i_brand_id#60, i_class_id#61, i_category_id#62, sales#68, number_sales#69] +Child 2 Input [6]: [channel#83, i_brand_id#76, i_class_id#77, i_category_id#78, sales#84, number_sales#85] + +(94) CometHashAggregate +Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] +Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] +Functions [2]: [partial_sum(sales#50), partial_sum(number_sales#51)] + +(95) CometColumnarExchange +Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#86, isEmpty#87, sum#88] +Arguments: hashpartitioning(channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(96) CometHashAggregate +Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#86, isEmpty#87, sum#88] +Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] +Functions [2]: [sum(sales#50), sum(number_sales#51)] + +(97) ReusedExchange [Reuses operator id: 95] +Output [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#86, isEmpty#87, sum#88] + +(98) CometHashAggregate +Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#86, isEmpty#87, sum#88] +Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] +Functions [2]: [sum(sales#50), sum(number_sales#51)] + +(99) CometHashAggregate +Input [5]: [channel#49, i_brand_id#40, i_class_id#41, sum_sales#89, number_sales#90] +Keys [3]: [channel#49, i_brand_id#40, i_class_id#41] +Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)] + +(100) CometColumnarExchange +Input [6]: [channel#49, i_brand_id#40, i_class_id#41, sum#91, isEmpty#92, sum#93] +Arguments: hashpartitioning(channel#49, i_brand_id#40, i_class_id#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(101) CometHashAggregate +Input [6]: [channel#49, i_brand_id#40, i_class_id#41, sum#91, isEmpty#92, sum#93] +Keys [3]: [channel#49, i_brand_id#40, i_class_id#41] +Functions [2]: [sum(sum_sales#89), sum(number_sales#90)] + +(102) ReusedExchange [Reuses operator id: 95] +Output [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#86, isEmpty#87, sum#88] + +(103) CometHashAggregate +Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#86, isEmpty#87, sum#88] +Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] +Functions [2]: [sum(sales#50), sum(number_sales#51)] + +(104) CometHashAggregate +Input [4]: [channel#49, i_brand_id#40, sum_sales#89, number_sales#90] +Keys [2]: [channel#49, i_brand_id#40] +Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)] + +(105) CometColumnarExchange +Input [5]: [channel#49, i_brand_id#40, sum#94, isEmpty#95, sum#96] +Arguments: hashpartitioning(channel#49, i_brand_id#40, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(106) CometHashAggregate +Input [5]: [channel#49, i_brand_id#40, sum#94, isEmpty#95, sum#96] +Keys [2]: [channel#49, i_brand_id#40] +Functions [2]: [sum(sum_sales#89), sum(number_sales#90)] + +(107) ReusedExchange [Reuses operator id: 95] +Output [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#86, isEmpty#87, sum#88] + +(108) CometHashAggregate +Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#86, isEmpty#87, sum#88] +Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] +Functions [2]: [sum(sales#50), sum(number_sales#51)] + +(109) CometHashAggregate +Input [3]: [channel#49, sum_sales#89, number_sales#90] +Keys [1]: [channel#49] +Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)] + +(110) CometColumnarExchange +Input [4]: [channel#49, sum#97, isEmpty#98, sum#99] +Arguments: hashpartitioning(channel#49, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(111) CometHashAggregate +Input [4]: [channel#49, sum#97, isEmpty#98, sum#99] +Keys [1]: [channel#49] +Functions [2]: [sum(sum_sales#89), sum(number_sales#90)] + +(112) ReusedExchange [Reuses operator id: 95] +Output [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#86, isEmpty#87, sum#88] + +(113) CometHashAggregate +Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#86, isEmpty#87, sum#88] +Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] +Functions [2]: [sum(sales#50), sum(number_sales#51)] + +(114) CometHashAggregate +Input [2]: [sum_sales#89, number_sales#90] Keys: [] -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] +Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)] -(119) Exchange -Input [3]: [sum#148, isEmpty#149, sum#150] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] +(115) CometColumnarExchange +Input [3]: [sum#100, isEmpty#101, sum#102] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(120) HashAggregate [codegen id : 194] -Input [3]: [sum#148, isEmpty#149, sum#150] +(116) CometHashAggregate +Input [3]: [sum#100, isEmpty#101, sum#102] Keys: [] -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 : 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 [2]: [sum(sum_sales#89), sum(number_sales#90)] + +(117) CometUnion +Child 0 Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90] +Child 1 Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#103, sum(sum_sales)#104, sum(number_sales)#105] +Child 2 Input [6]: [channel#49, i_brand_id#40, i_class_id#106, i_category_id#107, sum(sum_sales)#108, sum(number_sales)#109] +Child 3 Input [6]: [channel#49, i_brand_id#110, i_class_id#111, i_category_id#112, sum(sum_sales)#113, sum(number_sales)#114] +Child 4 Input [6]: [channel#115, i_brand_id#116, i_class_id#117, i_category_id#118, sum(sum_sales)#119, sum(number_sales)#120] + +(118) CometHashAggregate +Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90] +Keys [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90] Functions: [] -Aggregate Attributes: [] -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#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] +(119) CometColumnarExchange +Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90] +Arguments: hashpartitioning(channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(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] +(120) CometHashAggregate +Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90] +Keys [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90] Functions: [] -Aggregate Attributes: [] -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#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] +(121) CometTakeOrderedAndProject +Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#49 ASC NULLS FIRST,i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#49,i_brand_id#40,i_class_id#41,i_category_id#42,sum_sales#89,number_sales#90]), [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90], 100, [channel#49 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#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90] + +(122) ColumnarToRow [codegen id : 1] +Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90] ===== Subqueries ===== -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#159, ss_list_price#160, ss_sold_date_sk#161] +Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#52, [id=#53] +* ColumnarToRow (142) ++- CometHashAggregate (141) + +- CometColumnarExchange (140) + +- CometHashAggregate (139) + +- CometUnion (138) + :- CometProject (126) + : +- CometBroadcastHashJoin (125) + : :- CometScan parquet spark_catalog.default.store_sales (123) + : +- ReusedExchange (124) + :- CometProject (133) + : +- CometBroadcastHashJoin (132) + : :- CometScan parquet spark_catalog.default.catalog_sales (127) + : +- CometBroadcastExchange (131) + : +- CometProject (130) + : +- CometFilter (129) + : +- CometScan parquet spark_catalog.default.date_dim (128) + +- CometProject (137) + +- CometBroadcastHashJoin (136) + :- CometScan parquet spark_catalog.default.web_sales (134) + +- ReusedExchange (135) + + +(123) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_quantity#121, ss_list_price#122, ss_sold_date_sk#123] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#161), dynamicpruningexpression(ss_sold_date_sk#161 IN dynamicpruning#162)] +PartitionFilters: [isnotnull(ss_sold_date_sk#123), dynamicpruningexpression(ss_sold_date_sk#123 IN dynamicpruning#124)] ReadSchema: struct -(127) ReusedExchange [Reuses operator id: 21] -Output [1]: [d_date_sk#163] +(124) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#125] -(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 +(125) CometBroadcastHashJoin +Left output [3]: [ss_quantity#121, ss_list_price#122, ss_sold_date_sk#123] +Right output [1]: [d_date_sk#125] +Arguments: [ss_sold_date_sk#123], [d_date_sk#125], Inner, BuildRight -(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] +(126) CometProject +Input [4]: [ss_quantity#121, ss_list_price#122, ss_sold_date_sk#123, d_date_sk#125] +Arguments: [quantity#126, list_price#127], [ss_quantity#121 AS quantity#126, ss_list_price#122 AS list_price#127] -(130) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#166, cs_list_price#167, cs_sold_date_sk#168] +(127) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#128, cs_list_price#129, cs_sold_date_sk#130] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#168), dynamicpruningexpression(cs_sold_date_sk#168 IN dynamicpruning#169)] +PartitionFilters: [isnotnull(cs_sold_date_sk#130), dynamicpruningexpression(cs_sold_date_sk#130 IN dynamicpruning#131)] ReadSchema: struct -(131) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#170, d_year#171] +(128) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#132, d_year#133] 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#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)) +(129) CometFilter +Input [2]: [d_date_sk#132, d_year#133] +Condition : (((isnotnull(d_year#133) AND (d_year#133 >= 1998)) AND (d_year#133 <= 2000)) AND isnotnull(d_date_sk#132)) -(133) CometProject -Input [2]: [d_date_sk#170, d_year#171] -Arguments: [d_date_sk#170], [d_date_sk#170] +(130) CometProject +Input [2]: [d_date_sk#132, d_year#133] +Arguments: [d_date_sk#132], [d_date_sk#132] -(134) CometBroadcastExchange -Input [1]: [d_date_sk#170] -Arguments: [d_date_sk#170] +(131) CometBroadcastExchange +Input [1]: [d_date_sk#132] +Arguments: [d_date_sk#132] -(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 +(132) CometBroadcastHashJoin +Left output [3]: [cs_quantity#128, cs_list_price#129, cs_sold_date_sk#130] +Right output [1]: [d_date_sk#132] +Arguments: [cs_sold_date_sk#130], [d_date_sk#132], Inner, BuildRight -(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] +(133) CometProject +Input [4]: [cs_quantity#128, cs_list_price#129, cs_sold_date_sk#130, d_date_sk#132] +Arguments: [quantity#134, list_price#135], [cs_quantity#128 AS quantity#134, cs_list_price#129 AS list_price#135] -(137) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#174, ws_list_price#175, ws_sold_date_sk#176] +(134) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#136, ws_list_price#137, ws_sold_date_sk#138] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#176), dynamicpruningexpression(ws_sold_date_sk#176 IN dynamicpruning#177)] +PartitionFilters: [isnotnull(ws_sold_date_sk#138), dynamicpruningexpression(ws_sold_date_sk#138 IN dynamicpruning#139)] ReadSchema: struct -(138) ReusedExchange [Reuses operator id: 134] -Output [1]: [d_date_sk#178] +(135) ReusedExchange [Reuses operator id: 131] +Output [1]: [d_date_sk#140] -(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 +(136) CometBroadcastHashJoin +Left output [3]: [ws_quantity#136, ws_list_price#137, ws_sold_date_sk#138] +Right output [1]: [d_date_sk#140] +Arguments: [ws_sold_date_sk#138], [d_date_sk#140], Inner, BuildRight -(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] +(137) CometProject +Input [4]: [ws_quantity#136, ws_list_price#137, ws_sold_date_sk#138, d_date_sk#140] +Arguments: [quantity#141, list_price#142], [ws_quantity#136 AS quantity#141, ws_list_price#137 AS list_price#142] -(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] +(138) CometUnion +Child 0 Input [2]: [quantity#126, list_price#127] +Child 1 Input [2]: [quantity#134, list_price#135] +Child 2 Input [2]: [quantity#141, list_price#142] -(142) CometHashAggregate -Input [2]: [quantity#164, list_price#165] +(139) CometHashAggregate +Input [2]: [quantity#126, list_price#127] Keys: [] -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] +Functions [1]: [partial_avg((cast(quantity#126 as decimal(10,0)) * list_price#127))] -(144) Exchange -Input [2]: [sum#181, count#182] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=15] +(140) CometColumnarExchange +Input [2]: [sum#143, count#144] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(145) HashAggregate [codegen id : 2] -Input [2]: [sum#181, count#182] +(141) CometHashAggregate +Input [2]: [sum#143, count#144] Keys: [] -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] +Functions [1]: [avg((cast(quantity#126 as decimal(10,0)) * list_price#127))] + +(142) ColumnarToRow [codegen id : 1] +Input [1]: [average_sales#145] -Subquery:2 Hosting operator id = 126 Hosting Expression = ss_sold_date_sk#161 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 123 Hosting Expression = ss_sold_date_sk#123 IN dynamicpruning#12 -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) +Subquery:3 Hosting operator id = 127 Hosting Expression = cs_sold_date_sk#130 IN dynamicpruning#131 +BroadcastExchange (147) ++- * ColumnarToRow (146) + +- CometProject (145) + +- CometFilter (144) + +- CometScan parquet spark_catalog.default.date_dim (143) -(146) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#170, d_year#171] +(143) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#132, d_year#133] 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 -(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)) +(144) CometFilter +Input [2]: [d_date_sk#132, d_year#133] +Condition : (((isnotnull(d_year#133) AND (d_year#133 >= 1998)) AND (d_year#133 <= 2000)) AND isnotnull(d_date_sk#132)) -(148) CometProject -Input [2]: [d_date_sk#170, d_year#171] -Arguments: [d_date_sk#170], [d_date_sk#170] +(145) CometProject +Input [2]: [d_date_sk#132, d_year#133] +Arguments: [d_date_sk#132], [d_date_sk#132] -(149) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#170] +(146) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#132] -(150) BroadcastExchange -Input [1]: [d_date_sk#170] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] +(147) BroadcastExchange +Input [1]: [d_date_sk#132] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -Subquery:4 Hosting operator id = 137 Hosting Expression = ws_sold_date_sk#176 IN dynamicpruning#169 +Subquery:4 Hosting operator id = 134 Hosting Expression = ws_sold_date_sk#138 IN dynamicpruning#131 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (155) -+- * ColumnarToRow (154) - +- CometProject (153) - +- CometFilter (152) - +- CometScan parquet spark_catalog.default.date_dim (151) +BroadcastExchange (152) ++- * ColumnarToRow (151) + +- CometProject (150) + +- CometFilter (149) + +- CometScan parquet spark_catalog.default.date_dim (148) -(151) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#43, d_year#185, d_moy#186] +(148) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#43, d_year#44, d_moy#45] 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 -(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)) +(149) CometFilter +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) -(153) CometProject -Input [3]: [d_date_sk#43, d_year#185, d_moy#186] +(150) CometProject +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] Arguments: [d_date_sk#43], [d_date_sk#43] -(154) ColumnarToRow [codegen id : 1] +(151) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] -(155) BroadcastExchange +(152) BroadcastExchange Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (160) -+- * ColumnarToRow (159) - +- CometProject (158) - +- CometFilter (157) - +- CometScan parquet spark_catalog.default.date_dim (156) +Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 +BroadcastExchange (157) ++- * ColumnarToRow (156) + +- CometProject (155) + +- CometFilter (154) + +- CometScan parquet spark_catalog.default.date_dim (153) -(156) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#187] +(153) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_year#146] 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 -(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)) +(154) CometFilter +Input [2]: [d_date_sk#26, d_year#146] +Condition : (((isnotnull(d_year#146) AND (d_year#146 >= 1999)) AND (d_year#146 <= 2001)) AND isnotnull(d_date_sk#26)) -(158) CometProject -Input [2]: [d_date_sk#26, d_year#187] +(155) CometProject +Input [2]: [d_date_sk#26, d_year#146] Arguments: [d_date_sk#26], [d_date_sk#26] -(159) ColumnarToRow [codegen id : 1] +(156) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] -(160) BroadcastExchange +(157) BroadcastExchange Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] -Subquery:7 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 +Subquery:7 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#55, [id=#56] +Subquery:9 Hosting operator id = 78 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] -Subquery:10 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 65 Hosting Expression = cs_sold_date_sk#57 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#55, [id=#56] +Subquery:11 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] -Subquery:12 Hosting operator id = 82 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#73 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 7ef901a39..01803108f 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,240 +1,179 @@ -TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - 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 (195) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - InputAdapter - Union - 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 (37) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - InputAdapter - Union - WholeStageCodegen (12) - Filter [sales] - Subquery #3 - WholeStageCodegen (2) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - InputAdapter - Exchange #15 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [sum,count,quantity,list_price] - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,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_quantity,cs_list_price,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_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #17 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_quantity,ws_list_price,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 (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] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,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 #6 - 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 (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,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - CometBroadcastHashJoin [i_item_sk,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 [i_brand_id,i_class_id,i_category_id] #10 - CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [cs_item_sk,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - 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 [d_date_sk] #12 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #12 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ws_item_sk,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - ReusedExchange [d_date_sk] #12 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (9) - BroadcastHashJoin [i_item_sk,ss_item_sk] - 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 - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [d_date_sk] #4 - 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] #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] - Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - BroadcastHashJoin [cs_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 - InputAdapter - ReusedExchange [d_date_sk] #4 - 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] #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] - Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - BroadcastHashJoin [ws_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 - InputAdapter - ReusedExchange [d_date_sk] #4 - 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] #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 (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] #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] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] + CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] + CometUnion [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales,sum,isEmpty,sum,sum(sales),sum(number_salesL)] + CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id] #2 + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum,sales,number_sales] + CometUnion [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Subquery #3 + WholeStageCodegen (1) + ColumnarToRow InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - 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] #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 (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 #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 - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] + CometColumnarExchange #16 + CometHashAggregate [sum,count,quantity,list_price] + CometUnion [quantity,list_price] + CometProject [ss_quantity,ss_list_price] [quantity,list_price] + CometBroadcastHashJoin [ss_quantity,ss_list_price,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_quantity,cs_list_price,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 #17 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #18 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [ws_quantity,ws_list_price] [quantity,list_price] + CometBroadcastHashJoin [ws_quantity,ws_list_price,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] #18 + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #3 + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ss_quantity,ss_list_price] + CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk] #5 + CometProject [i_item_sk] [ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,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 [brand_id,class_id,category_id] #6 + CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + CometHashAggregate [brand_id,class_id,category_id] + CometColumnarExchange [brand_id,class_id,category_id] #7 + 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,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometBroadcastHashJoin [i_item_sk,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 [i_brand_id,i_class_id,i_category_id] #10 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + 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 [d_date_sk] #12 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #12 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #13 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + ReusedExchange [d_date_sk] #12 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] + 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] + ReusedExchange [ss_item_sk] #5 + CometBroadcastExchange [d_date_sk] #15 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + ReusedSubquery [average_sales] #3 + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1)] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #19 + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,cs_quantity,cs_list_price] + CometProject [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,ss_item_sk] + CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [ss_item_sk] #5 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 + ReusedExchange [d_date_sk] #15 + CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + ReusedSubquery [average_sales] #3 + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1)] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #20 + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ws_quantity,ws_list_price] + CometProject [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,ss_item_sk] + CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [ss_item_sk] #5 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 + ReusedExchange [d_date_sk] #15 + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum,sum(sum_sales),sum(number_salesL)] + CometColumnarExchange [channel,i_brand_id,i_class_id] #21 + CometHashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum,sum_sales,number_sales] + CometHashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales,i_category_id,sum,isEmpty,sum,sum(sales),sum(number_salesL)] + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum,sum(sum_sales),sum(number_salesL)] + CometColumnarExchange [channel,i_brand_id] #22 + CometHashAggregate [channel,i_brand_id,sum,isEmpty,sum,sum_sales,number_sales] + CometHashAggregate [channel,i_brand_id,sum_sales,number_sales,i_class_id,i_category_id,sum,isEmpty,sum,sum(sales),sum(number_salesL)] + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum,sum(sum_sales),sum(number_salesL)] + CometColumnarExchange [channel] #23 + CometHashAggregate [channel,sum,isEmpty,sum,sum_sales,number_sales] + CometHashAggregate [channel,sum_sales,number_sales,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum,sum(sales),sum(number_salesL)] + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum,sum(sum_sales),sum(number_salesL)] + CometColumnarExchange #24 + CometHashAggregate [sum,isEmpty,sum,sum_sales,number_sales] + CometHashAggregate [sum_sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum,sum(sales),sum(number_salesL)] + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 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 1ad55cbea..dab5b3d55 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,151 +1,147 @@ == Physical Plan == -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) +* ColumnarToRow (143) ++- CometTakeOrderedAndProject (142) + +- CometUnion (141) + :- CometHashAggregate (38) + : +- CometColumnarExchange (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) + :- CometHashAggregate (63) + : +- CometColumnarExchange (62) + : +- CometHashAggregate (61) + : +- CometProject (60) + : +- CometBroadcastHashJoin (59) + : :- CometProject (57) + : : +- CometBroadcastHashJoin (56) + : : :- CometProject (54) + : : : +- CometBroadcastHashJoin (53) + : : : :- CometProject (49) + : : : : +- CometBroadcastHashJoin (48) + : : : : :- CometProject (46) + : : : : : +- CometBroadcastHashJoin (45) + : : : : : :- CometProject (43) + : : : : : : +- CometBroadcastHashJoin (42) + : : : : : : :- CometFilter (40) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (39) + : : : : : : +- ReusedExchange (41) + : : : : : +- ReusedExchange (44) + : : : : +- ReusedExchange (47) + : : : +- CometBroadcastExchange (52) + : : : +- CometFilter (51) + : : : +- CometScan parquet spark_catalog.default.customer_address (50) + : : +- ReusedExchange (55) + : +- ReusedExchange (58) + :- CometHashAggregate (89) + : +- CometColumnarExchange (88) + : +- CometHashAggregate (87) + : +- CometProject (86) + : +- CometBroadcastHashJoin (85) + : :- CometProject (83) + : : +- CometBroadcastHashJoin (82) + : : :- CometProject (80) + : : : +- CometBroadcastHashJoin (79) + : : : :- CometProject (74) + : : : : +- CometBroadcastHashJoin (73) + : : : : :- CometProject (71) + : : : : : +- CometBroadcastHashJoin (70) + : : : : : :- CometProject (68) + : : : : : : +- CometBroadcastHashJoin (67) + : : : : : : :- CometFilter (65) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (64) + : : : : : : +- ReusedExchange (66) + : : : : : +- ReusedExchange (69) + : : : : +- ReusedExchange (72) + : : : +- CometBroadcastExchange (78) + : : : +- CometProject (77) + : : : +- CometFilter (76) + : : : +- CometScan parquet spark_catalog.default.customer_address (75) + : : +- ReusedExchange (81) + : +- ReusedExchange (84) + :- CometHashAggregate (115) + : +- CometColumnarExchange (114) + : +- CometHashAggregate (113) + : +- CometProject (112) + : +- CometBroadcastHashJoin (111) + : :- CometProject (109) + : : +- CometBroadcastHashJoin (108) + : : :- CometProject (106) + : : : +- CometBroadcastHashJoin (105) + : : : :- CometProject (100) + : : : : +- CometBroadcastHashJoin (99) + : : : : :- CometProject (97) + : : : : : +- CometBroadcastHashJoin (96) + : : : : : :- CometProject (94) + : : : : : : +- CometBroadcastHashJoin (93) + : : : : : : :- CometFilter (91) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (90) + : : : : : : +- ReusedExchange (92) + : : : : : +- ReusedExchange (95) + : : : : +- ReusedExchange (98) + : : : +- CometBroadcastExchange (104) + : : : +- CometProject (103) + : : : +- CometFilter (102) + : : : +- CometScan parquet spark_catalog.default.customer_address (101) + : : +- ReusedExchange (107) + : +- ReusedExchange (110) + +- CometHashAggregate (140) + +- CometColumnarExchange (139) + +- CometHashAggregate (138) + +- CometProject (137) + +- CometBroadcastHashJoin (136) + :- CometProject (132) + : +- CometBroadcastHashJoin (131) + : :- CometProject (129) + : : +- CometBroadcastHashJoin (128) + : : :- CometProject (126) + : : : +- CometBroadcastHashJoin (125) + : : : :- CometProject (123) + : : : : +- CometBroadcastHashJoin (122) + : : : : :- CometProject (120) + : : : : : +- CometBroadcastHashJoin (119) + : : : : : :- CometFilter (117) + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (116) + : : : : : +- ReusedExchange (118) + : : : : +- ReusedExchange (121) + : : : +- ReusedExchange (124) + : : +- ReusedExchange (127) + : +- ReusedExchange (130) + +- CometBroadcastExchange (135) + +- CometFilter (134) + +- CometScan parquet spark_catalog.default.item (133) (1) Scan parquet spark_catalog.default.catalog_sales @@ -321,532 +317,515 @@ Input [11]: [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, agg1#29, ag 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] +(37) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(38) CometHashAggregate 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 +(39) 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#64)] +PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#50)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(41) CometFilter +(40) 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)) -(42) ReusedExchange [Reuses operator id: 6] +(41) ReusedExchange [Reuses operator id: 6] Output [2]: [cd_demo_sk#11, cd_dep_count#14] -(43) CometBroadcastHashJoin +(42) 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 -(44) CometProject +(43) 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] -(45) ReusedExchange [Reuses operator id: 12] +(44) 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] -(46) CometBroadcastHashJoin +(45) 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 -(47) CometProject +(46) 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] -(48) ReusedExchange [Reuses operator id: 17] +(47) ReusedExchange [Reuses operator id: 17] Output [1]: [cd_demo_sk#20] -(49) CometBroadcastHashJoin +(48) 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 -(50) CometProject +(49) 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] -(51) Scan parquet spark_catalog.default.customer_address +(50) 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 -(52) CometFilter +(51) 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)) -(53) CometBroadcastExchange +(52) CometBroadcastExchange Input [3]: [ca_address_sk#21, ca_state#23, ca_country#24] Arguments: [ca_address_sk#21, ca_state#23, ca_country#24] -(54) CometBroadcastHashJoin +(53) 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 -(55) CometProject +(54) 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] -(56) ReusedExchange [Reuses operator id: 28] +(55) ReusedExchange [Reuses operator id: 28] Output [1]: [d_date_sk#25] -(57) CometBroadcastHashJoin +(56) 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 -(58) CometProject +(57) 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] +(58) ReusedExchange [Reuses operator id: 33] Output [2]: [i_item_sk#27, i_item_id#28] -(60) CometBroadcastHashJoin +(59) 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 +(60) 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 +(61) 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] +(62) CometColumnarExchange +Input [17]: [i_item_id#28, ca_country#24, ca_state#23, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64] +Arguments: hashpartitioning(i_item_id#28, ca_country#24, ca_state#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(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] +(63) CometHashAggregate +Input [17]: [i_item_id#28, ca_country#24, ca_state#23, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64] 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 +(64) 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#94)] +PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#65)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(67) CometFilter +(65) 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)) -(68) ReusedExchange [Reuses operator id: 6] +(66) ReusedExchange [Reuses operator id: 6] Output [2]: [cd_demo_sk#11, cd_dep_count#14] -(69) CometBroadcastHashJoin +(67) 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 -(70) CometProject +(68) 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] -(71) ReusedExchange [Reuses operator id: 12] +(69) 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] -(72) CometBroadcastHashJoin +(70) 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 -(73) CometProject +(71) 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] -(74) ReusedExchange [Reuses operator id: 17] +(72) ReusedExchange [Reuses operator id: 17] Output [1]: [cd_demo_sk#20] -(75) CometBroadcastHashJoin +(73) 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 -(76) CometProject +(74) 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] -(77) Scan parquet spark_catalog.default.customer_address +(75) 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 -(78) CometFilter +(76) 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)) -(79) CometProject +(77) 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] -(80) CometBroadcastExchange +(78) CometBroadcastExchange Input [2]: [ca_address_sk#21, ca_country#24] Arguments: [ca_address_sk#21, ca_country#24] -(81) CometBroadcastHashJoin +(79) 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 -(82) CometProject +(80) 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] -(83) ReusedExchange [Reuses operator id: 28] +(81) ReusedExchange [Reuses operator id: 28] Output [1]: [d_date_sk#25] -(84) CometBroadcastHashJoin +(82) 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 -(85) CometProject +(83) 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] +(84) ReusedExchange [Reuses operator id: 33] Output [2]: [i_item_sk#27, i_item_id#28] -(87) CometBroadcastHashJoin +(85) 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 +(86) 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 +(87) 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] +(88) CometColumnarExchange +Input [16]: [i_item_id#28, ca_country#24, sum#66, count#67, sum#68, count#69, sum#70, count#71, sum#72, count#73, sum#74, count#75, sum#76, count#77, sum#78, count#79] +Arguments: hashpartitioning(i_item_id#28, ca_country#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(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] - -(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] +(89) CometHashAggregate +Input [16]: [i_item_id#28, ca_country#24, sum#66, count#67, sum#68, count#69, sum#70, count#71, sum#72, count#73, sum#74, count#75, sum#76, count#77, sum#78, count#79] 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 +(90) 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#125)] +PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#80)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(94) CometFilter +(91) 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)) -(95) ReusedExchange [Reuses operator id: 6] +(92) ReusedExchange [Reuses operator id: 6] Output [2]: [cd_demo_sk#11, cd_dep_count#14] -(96) CometBroadcastHashJoin +(93) 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 -(97) CometProject +(94) 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] -(98) ReusedExchange [Reuses operator id: 12] +(95) 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] -(99) CometBroadcastHashJoin +(96) 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 -(100) CometProject +(97) 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] -(101) ReusedExchange [Reuses operator id: 17] +(98) ReusedExchange [Reuses operator id: 17] Output [1]: [cd_demo_sk#20] -(102) CometBroadcastHashJoin +(99) 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 -(103) CometProject +(100) 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] -(104) Scan parquet spark_catalog.default.customer_address +(101) 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 -(105) CometFilter +(102) 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)) -(106) CometProject +(103) CometProject Input [2]: [ca_address_sk#21, ca_state#23] Arguments: [ca_address_sk#21], [ca_address_sk#21] -(107) CometBroadcastExchange +(104) CometBroadcastExchange Input [1]: [ca_address_sk#21] Arguments: [ca_address_sk#21] -(108) CometBroadcastHashJoin +(105) 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 -(109) CometProject +(106) 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] -(110) ReusedExchange [Reuses operator id: 28] +(107) ReusedExchange [Reuses operator id: 28] Output [1]: [d_date_sk#25] -(111) CometBroadcastHashJoin +(108) 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 -(112) CometProject +(109) 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] +(110) ReusedExchange [Reuses operator id: 33] Output [2]: [i_item_sk#27, i_item_id#28] -(114) CometBroadcastHashJoin +(111) 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 +(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, 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] -(116) CometHashAggregate +(113) 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] +(114) CometColumnarExchange +Input [15]: [i_item_id#28, sum#81, count#82, sum#83, count#84, sum#85, count#86, sum#87, count#88, sum#89, count#90, sum#91, count#92, sum#93, count#94] +Arguments: hashpartitioning(i_item_id#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(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] +(115) CometHashAggregate +Input [15]: [i_item_id#28, sum#81, count#82, sum#83, count#84, sum#85, count#86, sum#87, count#88, sum#89, count#90, sum#91, count#92, sum#93, count#94] 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 +(116) 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#157)] +PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#95)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(121) CometFilter +(117) 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)) -(122) ReusedExchange [Reuses operator id: 6] +(118) ReusedExchange [Reuses operator id: 6] Output [2]: [cd_demo_sk#11, cd_dep_count#14] -(123) CometBroadcastHashJoin +(119) 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 -(124) CometProject +(120) 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] -(125) ReusedExchange [Reuses operator id: 12] +(121) 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] -(126) CometBroadcastHashJoin +(122) 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 -(127) CometProject +(123) 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] -(128) ReusedExchange [Reuses operator id: 17] +(124) ReusedExchange [Reuses operator id: 17] Output [1]: [cd_demo_sk#20] -(129) CometBroadcastHashJoin +(125) 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 -(130) CometProject +(126) 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] -(131) ReusedExchange [Reuses operator id: 107] +(127) ReusedExchange [Reuses operator id: 104] Output [1]: [ca_address_sk#21] -(132) CometBroadcastHashJoin +(128) 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 -(133) CometProject +(129) 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] -(134) ReusedExchange [Reuses operator id: 28] +(130) ReusedExchange [Reuses operator id: 28] Output [1]: [d_date_sk#25] -(135) CometBroadcastHashJoin +(131) 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 -(136) CometProject +(132) 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] -(137) Scan parquet spark_catalog.default.item +(133) 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 -(138) CometFilter +(134) CometFilter Input [1]: [i_item_sk#27] Condition : isnotnull(i_item_sk#27) -(139) CometBroadcastExchange +(135) CometBroadcastExchange Input [1]: [i_item_sk#27] Arguments: [i_item_sk#27] -(140) CometBroadcastHashJoin +(136) 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 -(141) CometProject +(137) 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] -(142) CometHashAggregate +(138) CometHashAggregate Input [7]: [agg1#29, agg2#30, agg3#31, agg4#32, agg5#33, agg6#34, agg7#35] Keys: [] 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] - -(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] +(139) CometColumnarExchange +Input [14]: [sum#96, count#97, sum#98, count#99, sum#100, count#101, sum#102, count#103, sum#104, count#105, sum#106, count#107, sum#108, count#109] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(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] +(140) CometHashAggregate +Input [14]: [sum#96, count#97, sum#98, count#99, sum#100, count#101, sum#102, count#103, sum#104, count#105, sum#106, count#107, sum#108, count#109] Keys: [] 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] -(146) Union +(141) CometUnion +Child 0 Input [11]: [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, agg1#110, agg2#111, agg3#112, agg4#113, agg5#114, agg6#115, agg7#116] +Child 1 Input [11]: [i_item_id#28, ca_country#24, ca_state#23, county#117, agg1#118, agg2#119, agg3#120, agg4#121, agg5#122, agg6#123, agg7#124] +Child 2 Input [11]: [i_item_id#28, ca_country#24, ca_state#125, county#126, agg1#127, agg2#128, agg3#129, agg4#130, agg5#131, agg6#132, agg7#133] +Child 3 Input [11]: [i_item_id#28, ca_country#134, ca_state#135, county#136, agg1#137, agg2#138, agg3#139, agg4#140, agg5#141, agg6#142, agg7#143] +Child 4 Input [11]: [i_item_id#144, ca_country#145, ca_state#146, county#147, agg1#148, agg2#149, agg3#150, agg4#151, agg5#152, agg6#153, agg7#154] + +(142) CometTakeOrderedAndProject +Input [11]: [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, agg1#110, agg2#111, agg3#112, agg4#113, agg5#114, agg6#115, agg7#116] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[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], output=[i_item_id#28,ca_country#24,ca_state#23,ca_county#22,agg1#110,agg2#111,agg3#112,agg4#113,agg5#114,agg6#115,agg7#116]), [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, agg1#110, agg2#111, agg3#112, agg4#113, agg5#114, agg6#115, agg7#116], 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#110, agg2#111, agg3#112, agg4#113, agg5#114, agg6#115, agg7#116] -(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] +(143) ColumnarToRow [codegen id : 1] +Input [11]: [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, agg1#110, agg2#111, agg3#112, agg4#113, agg5#114, agg6#115, agg7#116] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (152) -+- * ColumnarToRow (151) - +- CometProject (150) - +- CometFilter (149) - +- CometScan parquet spark_catalog.default.date_dim (148) +BroadcastExchange (148) ++- * ColumnarToRow (147) + +- CometProject (146) + +- CometFilter (145) + +- CometScan parquet spark_catalog.default.date_dim (144) -(148) Scan parquet spark_catalog.default.date_dim +(144) 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 -(149) CometFilter +(145) 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)) -(150) CometProject +(146) CometProject Input [2]: [d_date_sk#25, d_year#26] Arguments: [d_date_sk#25], [d_date_sk#25] -(151) ColumnarToRow [codegen id : 1] +(147) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] -(152) BroadcastExchange +(148) BroadcastExchange Input [1]: [d_date_sk#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:2 Hosting operator id = 39 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:3 Hosting operator id = 64 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:4 Hosting operator id = 90 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 +Subquery:5 Hosting operator id = 116 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 f854b2b31..aef60ba9b 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,179 +1,157 @@ -TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - Union - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,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,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,i_item_sk,i_item_id] - 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_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,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 [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,ca_address_sk,ca_county,ca_state,ca_country] - 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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - 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_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,cd_demo_sk,cd_dep_count] - CometFilter [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] - 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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange [cd_demo_sk] #5 - CometFilter [cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #8 - CometFilter [i_item_sk,i_item_id] - 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] #9 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,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,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country,i_item_sk,i_item_id] - 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_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,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 [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,ca_address_sk,ca_state,ca_country] - 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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - 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_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,cd_demo_sk,cd_dep_count] - CometFilter [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] - 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 - 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 [ca_address_sk,ca_state,ca_country] #10 - CometFilter [ca_address_sk,ca_state,ca_country] - 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] #11 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,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,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country,i_item_sk,i_item_id] - 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_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,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 [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,ca_address_sk,ca_country] - 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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - 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_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,cd_demo_sk,cd_dep_count] - CometFilter [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] - 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 - 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 [ca_address_sk,ca_country] #12 - CometProject [ca_address_sk,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] - 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] #13 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,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,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk,i_item_id] - 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_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,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 [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,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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - 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_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,cd_demo_sk,cd_dep_count] - CometFilter [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] - 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 - 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 [ca_address_sk] #14 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - 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 #15 - WholeStageCodegen (9) - ColumnarToRow - InputAdapter - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,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,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,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_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,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 [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,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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - 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_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,cd_demo_sk,cd_dep_count] - CometFilter [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] - 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 - 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 [i_item_sk] #16 - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometUnion [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [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,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] + CometColumnarExchange [i_item_id,ca_country,ca_state,ca_county] #1 + CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,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,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,i_item_sk,i_item_id] + 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_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,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 [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,ca_address_sk,ca_county,ca_state,ca_country] + 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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + 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_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,cd_demo_sk,cd_dep_count] + CometFilter [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] + 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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 + CometProject [cd_demo_sk,cd_dep_count] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometBroadcastExchange [cd_demo_sk] #5 + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] + CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] + CometBroadcastExchange [d_date_sk] #7 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_item_id] #8 + CometFilter [i_item_sk,i_item_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometHashAggregate [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,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] + CometColumnarExchange [i_item_id,ca_country,ca_state] #9 + CometHashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,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,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country,i_item_sk,i_item_id] + 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_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,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 [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,ca_address_sk,ca_state,ca_country] + 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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + 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_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,cd_demo_sk,cd_dep_count] + CometFilter [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] + 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 + 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 [ca_address_sk,ca_state,ca_country] #10 + CometFilter [ca_address_sk,ca_state,ca_country] + 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 + CometHashAggregate [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,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] + CometColumnarExchange [i_item_id,ca_country] #11 + CometHashAggregate [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,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,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country,i_item_sk,i_item_id] + 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_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,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 [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,ca_address_sk,ca_country] + 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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + 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_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,cd_demo_sk,cd_dep_count] + CometFilter [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] + 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 + 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 [ca_address_sk,ca_country] #12 + CometProject [ca_address_sk,ca_country] + CometFilter [ca_address_sk,ca_state,ca_country] + 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 + CometHashAggregate [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,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] + CometColumnarExchange [i_item_id] #13 + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,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,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk,i_item_id] + 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_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,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 [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,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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + 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_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,cd_demo_sk,cd_dep_count] + CometFilter [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] + 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 + 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 [ca_address_sk] #14 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + 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 + CometHashAggregate [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,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] + CometColumnarExchange #15 + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,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,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,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_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,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 [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,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 [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,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,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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + 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_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,cd_demo_sk,cd_dep_count] + CometFilter [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] + 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 + 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 [i_item_sk] #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 bfb24de43..e2ecb0b16 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 @@ -2,11 +2,11 @@ TakeOrderedAndProject (22) +- * Project (21) +- Window (20) - +- * Sort (19) - +- Exchange (18) - +- * HashAggregate (17) - +- Exchange (16) - +- * ColumnarToRow (15) + +- * ColumnarToRow (19) + +- CometSort (18) + +- CometColumnarExchange (17) + +- CometHashAggregate (16) + +- CometColumnarExchange (15) +- CometHashAggregate (14) +- CometProject (13) +- CometBroadcastHashJoin (12) @@ -92,39 +92,37 @@ Input [6]: [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, 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))] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(16) CometHashAggregate 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] -(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=2] +(17) CometColumnarExchange +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(18) CometSort +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] +Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15], [i_class#9 ASC NULLS FIRST] -(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 +(19) ColumnarToRow [codegen id : 1] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] (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] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] +Arguments: [sum(_w0#15) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#16], [i_class#9] -(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] +(21) Project [codegen id : 2] +Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, ((_w0#15 * 100) / _we0#16) AS revenueratio#17] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, _we0#16] (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] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] +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#17 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] ===== Subqueries ===== 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 e460ce5c9..7b17378dc 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,38 +1,34 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (4) + WholeStageCodegen (2) Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] InputAdapter Window [_w0,i_class] - WholeStageCodegen (3) - Sort [i_class] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [i_class] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,cs_ext_sales_price] - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometColumnarExchange [i_class] #1 + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum,sum(UnscaledValue(cs_ext_sales_price))] + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,cs_ext_sales_price] + CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + 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 2810779ed..1975ac486 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,28 +1,30 @@ == Physical Plan == -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) +TakeOrderedAndProject (26) ++- * HashAggregate (25) + +- * ColumnarToRow (24) + +- CometColumnarExchange (23) + +- RowToColumnar (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 @@ -124,50 +126,56 @@ Functions [1]: [partial_avg(inv_quantity_on_hand#2)] 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] -(22) Exchange +(22) RowToColumnar 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] -(23) HashAggregate [codegen id : 3] +(23) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] + +(24) ColumnarToRow [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] + +(25) 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)#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] -(24) TakeOrderedAndProject +(26) 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 (29) -+- * ColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan parquet spark_catalog.default.date_dim (25) +BroadcastExchange (31) ++- * ColumnarToRow (30) + +- CometProject (29) + +- CometFilter (28) + +- CometScan parquet spark_catalog.default.date_dim (27) -(25) Scan parquet spark_catalog.default.date_dim +(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 -(26) CometFilter +(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)) -(27) CometProject +(29) CometProject Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(28) ColumnarToRow [codegen id : 1] +(30) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(29) BroadcastExchange +(31) BroadcastExchange Input [1]: [d_date_sk#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-v2_7/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt index 0e864ab70..f5b281ac6 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,39 +1,41 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] 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 (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 - ColumnarToRow - InputAdapter - CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] - CometProject [inv_item_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.warehouse + ColumnarToRow + InputAdapter + CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 + RowToColumnar + 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 + ColumnarToRow + InputAdapter + CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] + CometProject [inv_item_sk,inv_quantity_on_hand] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + InputAdapter + 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 08bad9380..c4342ab4e 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,50 +1,50 @@ == Physical Plan == -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) +* ColumnarToRow (46) ++- CometTakeOrderedAndProject (45) + +- CometUnion (44) + :- CometHashAggregate (23) + : +- CometHashAggregate (22) + : +- CometHashAggregate (21) + : +- CometColumnarExchange (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) + :- CometHashAggregate (28) + : +- CometColumnarExchange (27) + : +- CometHashAggregate (26) + : +- CometHashAggregate (25) + : +- ReusedExchange (24) + :- CometHashAggregate (33) + : +- CometColumnarExchange (32) + : +- CometHashAggregate (31) + : +- CometHashAggregate (30) + : +- ReusedExchange (29) + :- CometHashAggregate (38) + : +- CometColumnarExchange (37) + : +- CometHashAggregate (36) + : +- CometHashAggregate (35) + : +- ReusedExchange (34) + +- CometHashAggregate (43) + +- CometColumnarExchange (42) + +- CometHashAggregate (41) + +- CometHashAggregate (40) + +- ReusedExchange (39) (1) Scan parquet spark_catalog.default.inventory @@ -140,151 +140,126 @@ Input [5]: [inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_prod Keys [4]: [i_product_name#12, i_brand#9, i_class#10, i_category#11] Functions [1]: [partial_avg(inv_quantity_on_hand#3)] -(20) ColumnarToRow [codegen id : 1] +(20) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(22) HashAggregate [codegen id : 2] +(21) CometHashAggregate 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)#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] +(22) CometHashAggregate +Input [5]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#16] 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] +Functions [1]: [partial_avg(qoh#16)] -(24) HashAggregate [codegen id : 2] -Input [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#20, count#21] +(23) CometHashAggregate +Input [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#17, count#18] 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] +Functions [1]: [avg(qoh#16)] -(25) ReusedExchange [Reuses operator id: 21] +(24) ReusedExchange [Reuses operator id: 20] Output [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#14, count#15] -(26) HashAggregate [codegen id : 4] +(25) CometHashAggregate 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)#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] +(26) CometHashAggregate +Input [4]: [i_product_name#12, i_brand#9, i_class#10, qoh#16] 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] +Functions [1]: [partial_avg(qoh#16)] -(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] +(27) CometColumnarExchange +Input [5]: [i_product_name#12, i_brand#9, i_class#10, sum#19, count#20] +Arguments: hashpartitioning(i_product_name#12, i_brand#9, i_class#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(29) HashAggregate [codegen id : 5] -Input [5]: [i_product_name#12, i_brand#9, i_class#10, sum#26, count#27] +(28) CometHashAggregate +Input [5]: [i_product_name#12, i_brand#9, i_class#10, sum#19, count#20] 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] +Functions [1]: [avg(qoh#16)] -(30) ReusedExchange [Reuses operator id: 21] +(29) ReusedExchange [Reuses operator id: 20] Output [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#14, count#15] -(31) HashAggregate [codegen id : 7] +(30) CometHashAggregate 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)#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] +(31) CometHashAggregate +Input [3]: [i_product_name#12, i_brand#9, qoh#16] 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] +Functions [1]: [partial_avg(qoh#16)] -(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] +(32) CometColumnarExchange +Input [4]: [i_product_name#12, i_brand#9, sum#21, count#22] +Arguments: hashpartitioning(i_product_name#12, i_brand#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(34) HashAggregate [codegen id : 8] -Input [4]: [i_product_name#12, i_brand#9, sum#33, count#34] +(33) CometHashAggregate +Input [4]: [i_product_name#12, i_brand#9, sum#21, count#22] 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] +Functions [1]: [avg(qoh#16)] -(35) ReusedExchange [Reuses operator id: 21] +(34) ReusedExchange [Reuses operator id: 20] Output [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#14, count#15] -(36) HashAggregate [codegen id : 10] +(35) CometHashAggregate 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)#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] +(36) CometHashAggregate +Input [2]: [i_product_name#12, qoh#16] 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] +Functions [1]: [partial_avg(qoh#16)] -(38) Exchange -Input [3]: [i_product_name#12, sum#41, count#42] -Arguments: hashpartitioning(i_product_name#12, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(37) CometColumnarExchange +Input [3]: [i_product_name#12, sum#23, count#24] +Arguments: hashpartitioning(i_product_name#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(39) HashAggregate [codegen id : 11] -Input [3]: [i_product_name#12, sum#41, count#42] +(38) CometHashAggregate +Input [3]: [i_product_name#12, sum#23, count#24] 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] +Functions [1]: [avg(qoh#16)] -(40) ReusedExchange [Reuses operator id: 21] +(39) ReusedExchange [Reuses operator id: 20] Output [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#14, count#15] -(41) HashAggregate [codegen id : 13] +(40) CometHashAggregate 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)#16] -Results [1]: [avg(inv_quantity_on_hand#3)#16 AS qoh#17] -(42) HashAggregate [codegen id : 13] -Input [1]: [qoh#17] +(41) CometHashAggregate +Input [1]: [qoh#16] Keys: [] -Functions [1]: [partial_avg(qoh#17)] -Aggregate Attributes [2]: [sum#48, count#49] -Results [2]: [sum#50, count#51] +Functions [1]: [partial_avg(qoh#16)] -(43) Exchange -Input [2]: [sum#50, count#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5] +(42) CometColumnarExchange +Input [2]: [sum#25, count#26] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(44) HashAggregate [codegen id : 14] -Input [2]: [sum#50, count#51] +(43) CometHashAggregate +Input [2]: [sum#25, count#26] Keys: [] -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] +Functions [1]: [avg(qoh#16)] + +(44) CometUnion +Child 0 Input [5]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#27] +Child 1 Input [5]: [i_product_name#12, i_brand#9, i_class#10, i_category#28, qoh#29] +Child 2 Input [5]: [i_product_name#12, i_brand#9, i_class#30, i_category#31, qoh#32] +Child 3 Input [5]: [i_product_name#12, i_brand#33, i_class#34, i_category#35, qoh#36] +Child 4 Input [5]: [i_product_name#37, i_brand#38, i_class#39, i_category#40, qoh#41] -(45) Union +(45) CometTakeOrderedAndProject +Input [5]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#27] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#27 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], output=[i_product_name#12,i_brand#9,i_class#10,i_category#11,qoh#27]), [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#27], 100, [qoh#27 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#27] -(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] +(46) ColumnarToRow [codegen id : 1] +Input [5]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#27] ===== Subqueries ===== 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 30307e528..d5a409c3e 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,74 +1,56 @@ -TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - Union - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count,inv_quantity_on_hand] - CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] - CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] - CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange [w_warehouse_sk] #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] #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 (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] #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 (11) - HashAggregate [i_product_name,sum,count] [avg(qoh),i_brand,i_class,i_category,qoh,sum,count] - InputAdapter - 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 (14) - HashAggregate [sum,count] [avg(qoh),i_product_name,i_brand,i_class,i_category,qoh,sum,count] - InputAdapter - 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 - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_product_name,i_brand,i_class,i_category,qoh] + CometUnion [i_product_name,i_brand,i_class,i_category,qoh] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh,sum,count,avg(qoh)] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count,qoh] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh,sum,count,avg(inv_quantity_on_hand)] + CometColumnarExchange [i_product_name,i_brand,i_class,i_category] #1 + CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count,inv_quantity_on_hand] + CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] + CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] + CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometBroadcastExchange [w_warehouse_sk] #5 + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh,sum,count,avg(qoh)] + CometColumnarExchange [i_product_name,i_brand,i_class] #6 + CometHashAggregate [i_product_name,i_brand,i_class,sum,count,qoh] + CometHashAggregate [i_product_name,i_brand,i_class,qoh,i_category,sum,count,avg(inv_quantity_on_hand)] + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh,sum,count,avg(qoh)] + CometColumnarExchange [i_product_name,i_brand] #7 + CometHashAggregate [i_product_name,i_brand,sum,count,qoh] + CometHashAggregate [i_product_name,i_brand,qoh,i_class,i_category,sum,count,avg(inv_quantity_on_hand)] + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh,sum,count,avg(qoh)] + CometColumnarExchange [i_product_name] #8 + CometHashAggregate [i_product_name,sum,count,qoh] + CometHashAggregate [i_product_name,qoh,i_brand,i_class,i_category,sum,count,avg(inv_quantity_on_hand)] + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh,sum,count,avg(qoh)] + CometColumnarExchange #9 + CometHashAggregate [sum,count,qoh] + CometHashAggregate [qoh,i_product_name,i_brand,i_class,i_category,sum,count,avg(inv_quantity_on_hand)] + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt index 8fd74c799..490d5cd18 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt @@ -1,52 +1,54 @@ == Physical Plan == -* Sort (48) -+- Exchange (47) - +- * Filter (46) - +- * HashAggregate (45) - +- Exchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- Exchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Project (14) - : : : : +- * SortMergeJoin Inner (13) - : : : : :- * Sort (6) - : : : : : +- Exchange (5) - : : : : : +- * ColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- * Sort (12) - : : : : +- Exchange (11) - : : : : +- * ColumnarToRow (10) - : : : : +- CometProject (9) - : : : : +- CometFilter (8) - : : : : +- CometScan parquet spark_catalog.default.store_returns (7) - : : : +- BroadcastExchange (19) - : : : +- * ColumnarToRow (18) - : : : +- CometProject (17) - : : : +- CometFilter (16) - : : : +- CometScan parquet spark_catalog.default.store (15) - : : +- BroadcastExchange (25) - : : +- * ColumnarToRow (24) - : : +- CometFilter (23) - : : +- CometScan parquet spark_catalog.default.item (22) - : +- BroadcastExchange (31) - : +- * ColumnarToRow (30) - : +- CometFilter (29) - : +- CometScan parquet spark_catalog.default.customer (28) - +- BroadcastExchange (37) - +- * ColumnarToRow (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.customer_address (34) +* ColumnarToRow (50) ++- CometSort (49) + +- CometColumnarExchange (48) + +- RowToColumnar (47) + +- * Filter (46) + +- * HashAggregate (45) + +- * ColumnarToRow (44) + +- CometColumnarExchange (43) + +- RowToColumnar (42) + +- * HashAggregate (41) + +- * HashAggregate (40) + +- * ColumnarToRow (39) + +- CometColumnarExchange (38) + +- RowToColumnar (37) + +- * HashAggregate (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * ColumnarToRow (29) + : +- CometProject (28) + : +- CometBroadcastHashJoin (27) + : :- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometProject (18) + : : : +- CometBroadcastHashJoin (17) + : : : :- CometProject (12) + : : : : +- CometSortMergeJoin (11) + : : : : :- CometSort (5) + : : : : : +- CometColumnarExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometSort (10) + : : : : +- CometColumnarExchange (9) + : : : : +- CometProject (8) + : : : : +- CometFilter (7) + : : : : +- CometScan parquet spark_catalog.default.store_returns (6) + : : : +- CometBroadcastExchange (16) + : : : +- CometProject (15) + : : : +- CometFilter (14) + : : : +- CometScan parquet spark_catalog.default.store (13) + : : +- CometBroadcastExchange (21) + : : +- CometFilter (20) + : : +- CometScan parquet spark_catalog.default.item (19) + : +- CometBroadcastExchange (26) + : +- CometFilter (25) + : +- CometScan parquet spark_catalog.default.customer (24) + +- BroadcastExchange (33) + +- * ColumnarToRow (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.customer_address (30) (1) Scan parquet spark_catalog.default.store_sales @@ -64,370 +66,384 @@ Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND is Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -(4) ColumnarToRow [codegen id : 1] +(4) CometColumnarExchange Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(5) Exchange +(5) CometSort Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] -(6) Sort [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST], false, 0 - -(7) Scan parquet spark_catalog.default.store_returns +(6) Scan parquet spark_catalog.default.store_returns Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(8) CometFilter +(7) CometFilter Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) -(9) CometProject +(8) CometProject Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] -(10) ColumnarToRow [codegen id : 3] +(9) CometColumnarExchange Input [2]: [sr_item_sk#7, sr_ticket_number#8] +Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(11) Exchange +(10) CometSort Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] -(12) Sort [codegen id : 4] -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST], false, 0 +(11) CometSortMergeJoin +Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Right output [2]: [sr_item_sk#7, sr_ticket_number#8] +Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner -(13) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_ticket_number#4, ss_item_sk#1] -Right keys [2]: [sr_ticket_number#8, sr_item_sk#7] -Join type: Inner -Join condition: None - -(14) Project [codegen id : 9] -Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +(12) CometProject Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -(15) Scan parquet spark_catalog.default.store +(13) Scan parquet spark_catalog.default.store Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(16) CometFilter +(14) CometFilter Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(s_zip#14)) -(17) CometProject +(15) CometProject Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Arguments: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14], [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(18) ColumnarToRow [codegen id : 5] +(16) CometBroadcastExchange Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(19) BroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(17) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +Right output [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight -(20) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#10] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 9] -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] +(18) CometProject Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] -(22) Scan parquet spark_catalog.default.item +(19) Scan parquet spark_catalog.default.item Output [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale ), IsNotNull(i_item_sk)] ReadSchema: struct -(23) CometFilter +(20) CometFilter Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Condition : ((isnotnull(i_color#18) AND (i_color#18 = pale )) AND isnotnull(i_item_sk#15)) -(24) ColumnarToRow [codegen id : 6] +(21) CometBroadcastExchange Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Arguments: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(25) BroadcastExchange -Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(22) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] +Right output [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Arguments: [ss_item_sk#1], [i_item_sk#15], Inner, BuildRight -(26) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#15] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 9] -Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +(23) CometProject Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(28) Scan parquet spark_catalog.default.customer +(24) Scan parquet spark_catalog.default.customer Output [5]: [c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] ReadSchema: struct -(29) CometFilter +(25) CometFilter Input [5]: [c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] Condition : ((isnotnull(c_customer_sk#21) AND isnotnull(c_current_addr_sk#22)) AND isnotnull(c_birth_country#25)) -(30) ColumnarToRow [codegen id : 7] -Input [5]: [c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] - -(31) BroadcastExchange +(26) CometBroadcastExchange Input [5]: [c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +Arguments: [c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] -(32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#21] -Join type: Inner -Join condition: None +(27) CometBroadcastHashJoin +Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Right output [5]: [c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] +Arguments: [ss_customer_sk#2], [c_customer_sk#21], Inner, BuildRight -(33) Project [codegen id : 9] -Output [13]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] +(28) CometProject Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] +Arguments: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25], [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] + +(29) ColumnarToRow [codegen id : 2] +Input [13]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] -(34) Scan parquet spark_catalog.default.customer_address +(30) Scan parquet spark_catalog.default.customer_address Output [4]: [ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct -(35) CometFilter +(31) CometFilter Input [4]: [ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] Condition : ((isnotnull(ca_address_sk#26) AND isnotnull(ca_country#29)) AND isnotnull(ca_zip#28)) -(36) ColumnarToRow [codegen id : 8] +(32) ColumnarToRow [codegen id : 1] Input [4]: [ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] -(37) BroadcastExchange +(33) BroadcastExchange Input [4]: [ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] -Arguments: HashedRelationBroadcastMode(List(input[0, int, false], upper(input[3, string, false]), input[2, string, false]),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(input[0, int, false], upper(input[3, string, false]), input[2, string, false]),false), [plan_id=3] -(38) BroadcastHashJoin [codegen id : 9] +(34) BroadcastHashJoin [codegen id : 2] Left keys [3]: [c_current_addr_sk#22, c_birth_country#25, s_zip#14] Right keys [3]: [ca_address_sk#26, upper(ca_country#29), ca_zip#28] Join type: Inner Join condition: None -(39) Project [codegen id : 9] +(35) Project [codegen id : 2] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, ca_state#27] Input [17]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25, ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] -(40) HashAggregate [codegen id : 9] +(36) HashAggregate [codegen id : 2] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, ca_state#27] Keys [10]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum#30] Results [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#31] -(41) Exchange +(37) RowToColumnar Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#31] -Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(42) HashAggregate [codegen id : 10] +(38) CometColumnarExchange +Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#31] +Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(39) ColumnarToRow [codegen id : 3] +Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#31] + +(40) HashAggregate [codegen id : 3] Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#31] Keys [10]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#32] Results [4]: [c_last_name#24, c_first_name#23, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#32,17,2) AS netpaid#33] -(43) HashAggregate [codegen id : 10] +(41) HashAggregate [codegen id : 3] Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, netpaid#33] Keys [3]: [c_last_name#24, c_first_name#23, s_store_name#11] Functions [1]: [partial_sum(netpaid#33)] Aggregate Attributes [2]: [sum#34, isEmpty#35] Results [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#36, isEmpty#37] -(44) Exchange +(42) RowToColumnar +Input [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#36, isEmpty#37] + +(43) CometColumnarExchange Input [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#36, isEmpty#37] -Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=8] +Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) HashAggregate [codegen id : 11] +(44) ColumnarToRow [codegen id : 4] +Input [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#36, isEmpty#37] + +(45) HashAggregate [codegen id : 4] Input [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#36, isEmpty#37] Keys [3]: [c_last_name#24, c_first_name#23, s_store_name#11] Functions [1]: [sum(netpaid#33)] Aggregate Attributes [1]: [sum(netpaid#33)#38] Results [4]: [c_last_name#24, c_first_name#23, s_store_name#11, sum(netpaid#33)#38 AS paid#39] -(46) Filter [codegen id : 11] +(46) Filter [codegen id : 4] Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] Condition : (isnotnull(paid#39) AND (cast(paid#39 as decimal(33,8)) > cast(Subquery scalar-subquery#40, [id=#41] as decimal(33,8)))) -(47) Exchange +(47) RowToColumnar +Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] + +(48) CometColumnarExchange Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] -Arguments: rangepartitioning(c_last_name#24 ASC NULLS FIRST, c_first_name#23 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Arguments: rangepartitioning(c_last_name#24 ASC NULLS FIRST, c_first_name#23 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(48) Sort [codegen id : 12] +(49) CometSort +Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] +Arguments: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39], [c_last_name#24 ASC NULLS FIRST, c_first_name#23 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST] + +(50) ColumnarToRow [codegen id : 5] Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] -Arguments: [c_last_name#24 ASC NULLS FIRST, c_first_name#23 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#40, [id=#41] -* HashAggregate (75) -+- Exchange (74) - +- * HashAggregate (73) - +- * HashAggregate (72) - +- Exchange (71) - +- * HashAggregate (70) - +- * Project (69) - +- * BroadcastHashJoin Inner BuildRight (68) - :- * Project (66) - : +- * BroadcastHashJoin Inner BuildRight (65) - : :- * Project (63) - : : +- * BroadcastHashJoin Inner BuildRight (62) - : : :- * Project (57) - : : : +- * BroadcastHashJoin Inner BuildRight (56) - : : : :- * Project (54) - : : : : +- * SortMergeJoin Inner (53) - : : : : :- * Sort (50) - : : : : : +- ReusedExchange (49) - : : : : +- * Sort (52) - : : : : +- ReusedExchange (51) - : : : +- ReusedExchange (55) - : : +- BroadcastExchange (61) - : : +- * ColumnarToRow (60) - : : +- CometFilter (59) - : : +- CometScan parquet spark_catalog.default.item (58) - : +- ReusedExchange (64) - +- ReusedExchange (67) - - -(49) ReusedExchange [Reuses operator id: 5] +* HashAggregate (81) ++- * ColumnarToRow (80) + +- CometColumnarExchange (79) + +- RowToColumnar (78) + +- * HashAggregate (77) + +- * HashAggregate (76) + +- * ColumnarToRow (75) + +- CometColumnarExchange (74) + +- RowToColumnar (73) + +- * HashAggregate (72) + +- * Project (71) + +- * BroadcastHashJoin Inner BuildRight (70) + :- * ColumnarToRow (68) + : +- CometProject (67) + : +- CometBroadcastHashJoin (66) + : :- CometProject (64) + : : +- CometBroadcastHashJoin (63) + : : :- CometProject (59) + : : : +- CometBroadcastHashJoin (58) + : : : :- CometProject (56) + : : : : +- CometSortMergeJoin (55) + : : : : :- CometSort (52) + : : : : : +- ReusedExchange (51) + : : : : +- CometSort (54) + : : : : +- ReusedExchange (53) + : : : +- ReusedExchange (57) + : : +- CometBroadcastExchange (62) + : : +- CometFilter (61) + : : +- CometScan parquet spark_catalog.default.item (60) + : +- ReusedExchange (65) + +- ReusedExchange (69) + + +(51) ReusedExchange [Reuses operator id: 4] Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -(50) Sort [codegen id : 2] +(52) CometSort Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST], false, 0 +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] -(51) ReusedExchange [Reuses operator id: 11] +(53) ReusedExchange [Reuses operator id: 9] Output [2]: [sr_item_sk#7, sr_ticket_number#8] -(52) Sort [codegen id : 4] +(54) CometSort Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST], false, 0 +Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] -(53) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_ticket_number#4, ss_item_sk#1] -Right keys [2]: [sr_ticket_number#8, sr_item_sk#7] -Join type: Inner -Join condition: None +(55) CometSortMergeJoin +Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Right output [2]: [sr_item_sk#7, sr_ticket_number#8] +Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner -(54) Project [codegen id : 9] -Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +(56) CometProject Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -(55) ReusedExchange [Reuses operator id: 19] +(57) ReusedExchange [Reuses operator id: 16] Output [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(56) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#10] -Join type: Inner -Join condition: None +(58) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +Right output [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight -(57) Project [codegen id : 9] -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] +(59) CometProject Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] -(58) Scan parquet spark_catalog.default.item +(60) Scan parquet spark_catalog.default.item Output [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(59) CometFilter +(61) CometFilter Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Condition : isnotnull(i_item_sk#15) -(60) ColumnarToRow [codegen id : 6] +(62) CometBroadcastExchange Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Arguments: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(61) BroadcastExchange -Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] +(63) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] +Right output [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Arguments: [ss_item_sk#1], [i_item_sk#15], Inner, BuildRight -(62) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#15] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 9] -Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +(64) CometProject Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(64) ReusedExchange [Reuses operator id: 31] +(65) ReusedExchange [Reuses operator id: 26] Output [5]: [c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] -(65) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#21] -Join type: Inner -Join condition: None +(66) CometBroadcastHashJoin +Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Right output [5]: [c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] +Arguments: [ss_customer_sk#2], [c_customer_sk#21], Inner, BuildRight -(66) Project [codegen id : 9] -Output [13]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] +(67) CometProject Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] +Arguments: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25], [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] -(67) ReusedExchange [Reuses operator id: 37] +(68) ColumnarToRow [codegen id : 2] +Input [13]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] + +(69) ReusedExchange [Reuses operator id: 33] Output [4]: [ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] -(68) BroadcastHashJoin [codegen id : 9] +(70) BroadcastHashJoin [codegen id : 2] Left keys [3]: [c_current_addr_sk#22, c_birth_country#25, s_zip#14] Right keys [3]: [ca_address_sk#26, upper(ca_country#29), ca_zip#28] Join type: Inner Join condition: None -(69) Project [codegen id : 9] +(71) Project [codegen id : 2] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, ca_state#27] Input [17]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25, ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] -(70) HashAggregate [codegen id : 9] +(72) HashAggregate [codegen id : 2] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, ca_state#27] Keys [10]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum#42] Results [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#43] -(71) Exchange +(73) RowToColumnar +Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#43] + +(74) CometColumnarExchange Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#43] -Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, [plan_id=11] +Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(72) HashAggregate [codegen id : 10] +(75) ColumnarToRow [codegen id : 3] +Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#43] + +(76) HashAggregate [codegen id : 3] Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#43] Keys [10]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#32] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#32,17,2) AS netpaid#33] -(73) HashAggregate [codegen id : 10] +(77) HashAggregate [codegen id : 3] Input [1]: [netpaid#33] Keys: [] Functions [1]: [partial_avg(netpaid#33)] Aggregate Attributes [2]: [sum#44, count#45] Results [2]: [sum#46, count#47] -(74) Exchange +(78) RowToColumnar +Input [2]: [sum#46, count#47] + +(79) CometColumnarExchange +Input [2]: [sum#46, count#47] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(80) ColumnarToRow [codegen id : 4] Input [2]: [sum#46, count#47] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] -(75) HashAggregate [codegen id : 11] +(81) HashAggregate [codegen id : 4] Input [2]: [sum#46, count#47] Keys: [] Functions [1]: [avg(netpaid#33)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt index 5b658f1d7..42c283018 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt @@ -1,122 +1,100 @@ -WholeStageCodegen (12) - Sort [c_last_name,c_first_name,s_store_name] +WholeStageCodegen (5) + ColumnarToRow InputAdapter - Exchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (11) - Filter [paid] - Subquery #1 - WholeStageCodegen (11) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - InputAdapter - Exchange #10 - WholeStageCodegen (10) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - InputAdapter - Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 - WholeStageCodegen (9) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - WholeStageCodegen (2) - Sort [ss_ticket_number,ss_item_sk] - InputAdapter - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #4 - InputAdapter - WholeStageCodegen (4) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - ReusedExchange [sr_item_sk,sr_ticket_number] #5 - InputAdapter - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #6 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - InputAdapter - ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 - InputAdapter - ReusedExchange [ca_address_sk,ca_state,ca_zip,ca_country] #9 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - InputAdapter - Exchange [c_last_name,c_first_name,s_store_name] #2 - WholeStageCodegen (10) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + CometSort [c_last_name,c_first_name,s_store_name,paid] + CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 + RowToColumnar + WholeStageCodegen (4) + Filter [paid] + Subquery #1 + WholeStageCodegen (4) + HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] + ColumnarToRow InputAdapter - Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 - WholeStageCodegen (9) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - WholeStageCodegen (2) - Sort [ss_ticket_number,ss_item_sk] - InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + CometColumnarExchange #10 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [netpaid] [sum,count,sum,count] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] + ColumnarToRow + InputAdapter + CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] + CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] + ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #4 + CometSort [sr_item_sk,sr_ticket_number] + ReusedExchange [sr_item_sk,sr_ticket_number] #5 + ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #6 + CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #12 + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 InputAdapter - WholeStageCodegen (4) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - Exchange [sr_ticket_number,sr_item_sk] #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + ReusedExchange [ca_address_sk,ca_state,ca_zip,ca_country] #9 + HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [c_last_name,c_first_name,s_store_name] #2 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] + ColumnarToRow + InputAdapter + CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] + CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #4 + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + CometSort [sr_item_sk,sr_ticket_number] + CometColumnarExchange [sr_ticket_number,sr_item_sk] #5 + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #6 + CometProject [s_store_sk,s_store_name,s_state,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #7 + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) + BroadcastExchange #9 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] + CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] 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 4abfc188b..8d5df655c 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,75 +1,73 @@ == Physical Plan == -TakeOrderedAndProject (71) -+- Union (70) - :- * HashAggregate (28) - : +- Exchange (27) - : +- * 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 (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) +* ColumnarToRow (69) ++- CometTakeOrderedAndProject (68) + +- CometUnion (67) + :- CometHashAggregate (27) + : +- CometColumnarExchange (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) + :- CometHashAggregate (47) + : +- CometColumnarExchange (46) + : +- CometHashAggregate (45) + : +- CometProject (44) + : +- CometBroadcastHashJoin (43) + : :- CometProject (41) + : : +- CometBroadcastHashJoin (40) + : : :- CometProject (35) + : : : +- CometBroadcastHashJoin (34) + : : : :- CometProject (32) + : : : : +- CometBroadcastHashJoin (31) + : : : : :- CometFilter (29) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (28) + : : : : +- ReusedExchange (30) + : : : +- ReusedExchange (33) + : : +- CometBroadcastExchange (39) + : : +- CometProject (38) + : : +- CometFilter (37) + : : +- CometScan parquet spark_catalog.default.store (36) + : +- ReusedExchange (42) + +- CometHashAggregate (66) + +- CometColumnarExchange (65) + +- CometHashAggregate (64) + +- CometProject (63) + +- CometBroadcastHashJoin (62) + :- CometProject (58) + : +- CometBroadcastHashJoin (57) + : :- CometProject (55) + : : +- CometBroadcastHashJoin (54) + : : :- CometProject (52) + : : : +- CometBroadcastHashJoin (51) + : : : :- CometFilter (49) + : : : : +- CometScan parquet spark_catalog.default.store_sales (48) + : : : +- ReusedExchange (50) + : : +- ReusedExchange (53) + : +- ReusedExchange (56) + +- CometBroadcastExchange (61) + +- CometFilter (60) + +- CometScan parquet spark_catalog.default.item (59) (1) Scan parquet spark_catalog.default.store_sales @@ -193,246 +191,237 @@ 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) ColumnarToRow [codegen id : 1] +(26) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(27) Exchange -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 : 2] +(27) CometHashAggregate 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 +(28) 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#41)] +PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#32)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(30) CometFilter +(29) 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)) -(31) ReusedExchange [Reuses operator id: 6] +(30) ReusedExchange [Reuses operator id: 6] Output [1]: [cd_demo_sk#10] -(32) CometBroadcastHashJoin +(31) 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 -(33) CometProject +(32) 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] -(34) ReusedExchange [Reuses operator id: 12] +(33) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#14] -(35) CometBroadcastHashJoin +(34) 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 -(36) CometProject +(35) 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] -(37) Scan parquet spark_catalog.default.store +(36) 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 -(38) CometFilter +(37) 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 +(38) CometProject Input [2]: [s_store_sk#16, s_state#17] Arguments: [s_store_sk#16], [s_store_sk#16] -(40) CometBroadcastExchange +(39) CometBroadcastExchange Input [1]: [s_store_sk#16] Arguments: [s_store_sk#16] -(41) CometBroadcastHashJoin +(40) 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 +(41) 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] +(42) ReusedExchange [Reuses operator id: 22] Output [2]: [i_item_sk#18, i_item_id#19] -(44) CometBroadcastHashJoin +(43) 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 +(44) 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 +(45) 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] +(46) CometColumnarExchange +Input [9]: [i_item_id#19, sum#33, count#34, sum#35, count#36, sum#37, count#38, sum#39, count#40] +Arguments: hashpartitioning(i_item_id#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(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] +(47) CometHashAggregate +Input [9]: [i_item_id#19, sum#33, count#34, sum#35, count#36, sum#37, count#38, sum#39, count#40] 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 +(48) 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#60)] +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 -(51) CometFilter +(49) 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)) -(52) ReusedExchange [Reuses operator id: 6] +(50) ReusedExchange [Reuses operator id: 6] Output [1]: [cd_demo_sk#10] -(53) CometBroadcastHashJoin +(51) 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 -(54) CometProject +(52) 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] -(55) ReusedExchange [Reuses operator id: 12] +(53) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#14] -(56) CometBroadcastHashJoin +(54) 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 -(57) CometProject +(55) 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] -(58) ReusedExchange [Reuses operator id: 40] +(56) ReusedExchange [Reuses operator id: 39] Output [1]: [s_store_sk#16] -(59) CometBroadcastHashJoin +(57) 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 -(60) CometProject +(58) 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] -(61) Scan parquet spark_catalog.default.item +(59) 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 -(62) CometFilter +(60) CometFilter Input [1]: [i_item_sk#18] Condition : isnotnull(i_item_sk#18) -(63) CometBroadcastExchange +(61) CometBroadcastExchange Input [1]: [i_item_sk#18] Arguments: [i_item_sk#18] -(64) CometBroadcastHashJoin +(62) 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 -(65) CometProject +(63) 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] -(66) CometHashAggregate +(64) CometHashAggregate Input [4]: [agg1#20, agg2#21, agg3#22, agg4#23] Keys: [] 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] - -(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] +(65) CometColumnarExchange +Input [8]: [sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(69) HashAggregate [codegen id : 6] -Input [8]: [sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] +(66) CometHashAggregate +Input [8]: [sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49] Keys: [] 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] -(70) Union +(67) CometUnion +Child 0 Input [7]: [i_item_id#19, s_state#17, g_state#50, agg1#51, agg2#52, agg3#53, agg4#54] +Child 1 Input [7]: [i_item_id#19, s_state#55, g_state#56, agg1#57, agg2#58, agg3#59, agg4#60] +Child 2 Input [7]: [i_item_id#61, s_state#62, g_state#63, agg1#64, agg2#65, agg3#66, agg4#67] + +(68) CometTakeOrderedAndProject +Input [7]: [i_item_id#19, s_state#17, g_state#50, agg1#51, agg2#52, agg3#53, agg4#54] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#19 ASC NULLS FIRST,s_state#17 ASC NULLS FIRST], output=[i_item_id#19,s_state#17,g_state#50,agg1#51,agg2#52,agg3#53,agg4#54]), [i_item_id#19, s_state#17, g_state#50, agg1#51, agg2#52, agg3#53, agg4#54], 100, [i_item_id#19 ASC NULLS FIRST, s_state#17 ASC NULLS FIRST], [i_item_id#19, s_state#17, g_state#50, agg1#51, agg2#52, agg3#53, agg4#54] -(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] +(69) ColumnarToRow [codegen id : 1] +Input [7]: [i_item_id#19, s_state#17, g_state#50, agg1#51, agg2#52, agg3#53, agg4#54] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 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 (74) ++- * ColumnarToRow (73) + +- CometProject (72) + +- CometFilter (71) + +- CometScan parquet spark_catalog.default.date_dim (70) -(72) Scan parquet spark_catalog.default.date_dim +(70) 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 -(73) CometFilter +(71) 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)) -(74) CometProject +(72) CometProject Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(75) ColumnarToRow [codegen id : 1] +(73) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(76) BroadcastExchange +(74) BroadcastExchange Input [1]: [d_date_sk#14] 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:2 Hosting operator id = 28 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 +Subquery:3 Hosting operator id = 48 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 de75d46ee..43f9cbc6e 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,93 +1,81 @@ -TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - Union - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count,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,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] - CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_state] #5 - CometFilter [s_store_sk,s_state] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometFilter [i_item_sk,i_item_id] - 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] #7 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,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,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] - CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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 - ReusedExchange [cd_demo_sk] #3 - ReusedExchange [d_date_sk] #4 - CometBroadcastExchange [s_store_sk] #8 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - 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 #9 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] - CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk] - CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] - CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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 - ReusedExchange [cd_demo_sk] #3 - ReusedExchange [d_date_sk] #4 - ReusedExchange [s_store_sk] #8 - CometBroadcastExchange [i_item_sk] #10 - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] + CometUnion [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] + CometHashAggregate [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] + CometColumnarExchange [i_item_id,s_state] #1 + CometHashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count,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,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id] + CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [cd_demo_sk] #3 + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [s_store_sk,s_state] #5 + CometFilter [s_store_sk,s_state] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometHashAggregate [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] + CometColumnarExchange [i_item_id] #7 + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,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,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] + CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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 + ReusedExchange [cd_demo_sk] #3 + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange [s_store_sk] #8 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + ReusedExchange [i_item_sk,i_item_id] #6 + CometHashAggregate [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] + CometColumnarExchange #9 + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] + CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [agg1,agg2,agg3,agg4] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk] + CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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 + ReusedExchange [cd_demo_sk] #3 + ReusedExchange [d_date_sk] #4 + ReusedExchange [s_store_sk] #8 + CometBroadcastExchange [i_item_sk] #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 92e11aabe..d978de885 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,37 +1,36 @@ == Physical Plan == -* 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) +* ColumnarToRow (32) ++- CometSort (31) + +- CometColumnarExchange (30) + +- CometProject (29) + +- CometBroadcastHashJoin (28) + :- CometFilter (24) + : +- CometHashAggregate (23) + : +- CometColumnarExchange (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) + +- CometBroadcastExchange (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.customer (25) (1) Scan parquet spark_catalog.default.store_sales @@ -135,90 +134,84 @@ 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)] -(22) ColumnarToRow [codegen id : 1] +(22) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(24) HashAggregate [codegen id : 3] +(23) CometHashAggregate 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)#17] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#17 AS cnt#18] -(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)) +(24) CometFilter +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] +Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) -(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] +(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] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(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) +(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) -(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] +(27) CometBroadcastExchange +Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Arguments: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -(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] +(28) CometBroadcastHashJoin +Left output [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] +Right output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight -(30) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#19] -Join type: Inner -Join condition: None +(29) CometProject +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] +Arguments: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17], [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -(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] +(30) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=2] -(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] +(31) CometSort +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, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17], [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] -(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 +(32) ColumnarToRow [codegen id : 1] +Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (38) -+- * ColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.date_dim (34) +BroadcastExchange (37) ++- * ColumnarToRow (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) -(34) Scan parquet spark_catalog.default.date_dim +(33) 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 -(35) CometFilter +(34) 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)) -(36) CometProject +(35) CometProject Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(37) ColumnarToRow [codegen id : 1] +(36) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(38) BroadcastExchange +(37) BroadcastExchange Input [1]: [d_date_sk#7] -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/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt index 814e543ed..fafc612d6 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,50 +1,42 @@ -WholeStageCodegen (4) - Sort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] +WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,count] - CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] - CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] - CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #5 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange [hd_demo_sk] #6 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 + CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometFilter [ss_ticket_number,ss_customer_sk,cnt] + CometHashAggregate [ss_ticket_number,ss_customer_sk,cnt,count,count(1)] + CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] + CometProject [ss_customer_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange [s_store_sk] #5 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_county] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county] + CometBroadcastExchange [hd_demo_sk] #6 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] #7 + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] 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 790000085..721036bfb 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,48 +1,50 @@ == Physical Plan == -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) +TakeOrderedAndProject (46) ++- * HashAggregate (45) + +- * ColumnarToRow (44) + +- CometColumnarExchange (43) + +- RowToColumnar (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 @@ -238,50 +240,56 @@ Functions [10]: [partial_count(1), partial_avg(cd_dep_count#25), partial_max(cd_ 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] -(42) Exchange +(42) RowToColumnar 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] -(43) HashAggregate [codegen id : 6] +(43) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=5] + +(44) ColumnarToRow [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] + +(45) 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] -(44) TakeOrderedAndProject +(46) 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 = 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) +BroadcastExchange (51) ++- * ColumnarToRow (50) + +- CometProject (49) + +- CometFilter (48) + +- CometScan parquet spark_catalog.default.date_dim (47) -(45) Scan parquet spark_catalog.default.date_dim +(47) 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 -(46) CometFilter +(48) 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)) -(47) CometProject +(49) CometProject Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(48) ColumnarToRow [codegen id : 1] +(50) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(49) BroadcastExchange +(51) BroadcastExchange Input [1]: [d_date_sk#9] 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/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt index 0b55e23ab..221648657 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,70 +1,72 @@ 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 (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 (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] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] - CometFilter [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] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,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 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + RowToColumnar + 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] + Project [c_current_cdemo_sk,ca_state] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] + Filter [exists,exists] + BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,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 + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [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] + CometBroadcastExchange [ss_customer_sk] #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,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 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [cs_ship_customer_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,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 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) + BroadcastExchange #8 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - 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] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + 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/q35a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt index 215d4240e..476dab310 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,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (38) -+- * HashAggregate (37) - +- Exchange (36) - +- * ColumnarToRow (35) +* ColumnarToRow (38) ++- CometTakeOrderedAndProject (37) + +- CometHashAggregate (36) + +- CometColumnarExchange (35) +- CometHashAggregate (34) +- CometProject (33) +- CometBroadcastHashJoin (32) @@ -202,23 +202,21 @@ Input [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd 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] +(35) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(36) CometHashAggregate 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] +(37) CometTakeOrderedAndProject +Input [18]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#41, avg(cd_dep_count)#42, max(cd_dep_count)#43, sum(cd_dep_count)#44, cd_dep_employed_count#26, cnt2#45, avg(cd_dep_employed_count)#46, max(cd_dep_employed_count)#47, sum(cd_dep_employed_count)#48, cd_dep_college_count#27, cnt3#49, avg(cd_dep_college_count)#50, max(cd_dep_college_count)#51, sum(cd_dep_college_count)#52] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[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], output=[ca_state#21,cd_gender#23,cd_marital_status#24,cd_dep_count#25,cnt1#41,avg(cd_dep_count)#42,max(cd_dep_count)#43,sum(cd_dep_count)#44,cd_dep_employed_count#26,cnt2#45,avg(cd_dep_employed_count)#46,max(cd_dep_employed_count)#47,sum(cd_dep_employed_count)#48,cd_dep_college_count#27,cnt3#49,avg(cd_dep_college_count)#50,max(cd_dep_college_count)#51,sum(cd_dep_college_count)#52]), [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#41, avg(cd_dep_count)#42, max(cd_dep_count)#43, sum(cd_dep_count)#44, cd_dep_employed_count#26, cnt2#45, avg(cd_dep_employed_count)#46, max(cd_dep_employed_count)#47, sum(cd_dep_employed_count)#48, cd_dep_college_count#27, cnt3#49, avg(cd_dep_college_count)#50, max(cd_dep_college_count)#51, sum(cd_dep_college_count)#52], 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#41, avg(cd_dep_count)#42, max(cd_dep_count)#43, sum(cd_dep_count)#44, cd_dep_employed_count#26, cnt2#45, avg(cd_dep_employed_count)#46, max(cd_dep_employed_count)#47, sum(cd_dep_employed_count)#48, cd_dep_college_count#27, cnt3#49, avg(cd_dep_college_count)#50, max(cd_dep_college_count)#51, sum(cd_dep_college_count)#52] + +(38) ColumnarToRow [codegen id : 1] +Input [18]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#41, avg(cd_dep_count)#42, max(cd_dep_count)#43, sum(cd_dep_count)#44, cd_dep_employed_count#26, cnt2#45, avg(cd_dep_employed_count)#46, max(cd_dep_employed_count)#47, sum(cd_dep_employed_count)#48, cd_dep_college_count#27, cnt3#49, avg(cd_dep_college_count)#50, max(cd_dep_college_count)#51, sum(cd_dep_college_count)#52] ===== Subqueries ===== 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 8d73022ee..25f72e37b 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,52 +1,50 @@ -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 (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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [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] - CometProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometBroadcastHashJoin [c_current_cdemo_sk,ca_state,cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometProject [c_current_cdemo_sk,ca_state] - CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk,ca_state] - CometProject [c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customsk] - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] - CometFilter [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] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [customsk] #5 - CometUnion [customsk] - CometProject [ws_bill_customer_sk] [customsk] - CometBroadcastHashJoin [ws_bill_customer_sk,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_ship_customer_sk,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 [ca_address_sk,ca_state] #6 - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - 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] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cd_dep_employed_count,cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cd_dep_college_count,cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] + CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cd_dep_employed_count,cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cd_dep_college_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,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)] + CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + CometHashAggregate [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] + CometProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometBroadcastHashJoin [c_current_cdemo_sk,ca_state,cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometProject [c_current_cdemo_sk,ca_state] + CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk,ca_state] + CometProject [c_current_cdemo_sk,c_current_addr_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customsk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [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] + CometBroadcastExchange [ss_customer_sk] #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,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_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [customsk] #5 + CometUnion [customsk] + CometProject [ws_bill_customer_sk] [customsk] + CometBroadcastHashJoin [ws_bill_customer_sk,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_ship_customer_sk,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 [ca_address_sk,ca_state] #6 + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + 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 3bdded0ed..149347c76 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 @@ -2,45 +2,45 @@ 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) + +- * ColumnarToRow (39) + +- CometSort (38) + +- CometColumnarExchange (37) + +- CometHashAggregate (36) + +- CometColumnarExchange (35) + +- CometHashAggregate (34) + +- CometUnion (33) + :- CometHashAggregate (22) + : +- CometColumnarExchange (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) + :- CometHashAggregate (27) + : +- CometColumnarExchange (26) + : +- CometHashAggregate (25) + : +- CometHashAggregate (24) + : +- ReusedExchange (23) + +- CometHashAggregate (32) + +- CometColumnarExchange (31) + +- CometHashAggregate (30) + +- CometHashAggregate (29) + +- ReusedExchange (28) (1) Scan parquet spark_catalog.default.store_sales @@ -140,115 +140,100 @@ 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))] -(21) ColumnarToRow [codegen id : 1] +(21) CometColumnarExchange Input [4]: [i_category#11, i_class#10, sum#14, sum#15] +Arguments: hashpartitioning(i_category#11, i_class#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(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] - -(23) HashAggregate [codegen id : 2] +(22) CometHashAggregate 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))#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] -(24) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#11, i_class#10, sum#22, sum#23] +(23) ReusedExchange [Reuses operator id: 21] +Output [4]: [i_category#11, i_class#10, sum#16, sum#17] -(25) HashAggregate [codegen id : 4] -Input [4]: [i_category#11, i_class#10, sum#22, sum#23] +(24) CometHashAggregate +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))#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] +(25) CometHashAggregate +Input [3]: [ss_net_profit#18, ss_ext_sales_price#19, 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] +Functions [2]: [partial_sum(ss_net_profit#18), partial_sum(ss_ext_sales_price#19)] -(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] +(26) CometColumnarExchange +Input [5]: [i_category#11, sum#20, isEmpty#21, sum#22, isEmpty#23] +Arguments: hashpartitioning(i_category#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(28) HashAggregate [codegen id : 5] -Input [5]: [i_category#11, sum#32, isEmpty#33, sum#34, isEmpty#35] +(27) CometHashAggregate +Input [5]: [i_category#11, sum#20, isEmpty#21, sum#22, isEmpty#23] 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] +Functions [2]: [sum(ss_net_profit#18), sum(ss_ext_sales_price#19)] -(29) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#11, i_class#10, sum#43, sum#44] +(28) ReusedExchange [Reuses operator id: 21] +Output [4]: [i_category#11, i_class#10, sum#24, sum#25] -(30) HashAggregate [codegen id : 7] -Input [4]: [i_category#11, i_class#10, sum#43, sum#44] +(29) CometHashAggregate +Input [4]: [i_category#11, i_class#10, sum#24, sum#25] 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))#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] -(31) HashAggregate [codegen id : 7] -Input [2]: [ss_net_profit#26, ss_ext_sales_price#27] +(30) CometHashAggregate +Input [2]: [ss_net_profit#18, ss_ext_sales_price#19] Keys: [] -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] +Functions [2]: [partial_sum(ss_net_profit#18), partial_sum(ss_ext_sales_price#19)] -(32) Exchange -Input [4]: [sum#49, isEmpty#50, sum#51, isEmpty#52] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] +(31) CometColumnarExchange +Input [4]: [sum#26, isEmpty#27, sum#28, isEmpty#29] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(33) HashAggregate [codegen id : 8] -Input [4]: [sum#49, isEmpty#50, sum#51, isEmpty#52] +(32) CometHashAggregate +Input [4]: [sum#26, isEmpty#27, sum#28, isEmpty#29] Keys: [] -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] +Functions [2]: [sum(ss_net_profit#18), sum(ss_ext_sales_price#19)] -(34) Union +(33) CometUnion +Child 0 Input [6]: [gross_margin#30, i_category#11, i_class#10, t_category#31, t_class#32, lochierarchy#33] +Child 1 Input [6]: [gross_margin#34, i_category#11, i_class#35, t_category#36, t_class#37, lochierarchy#38] +Child 2 Input [6]: [gross_margin#39, i_category#40, i_class#41, t_category#42, t_class#43, lochierarchy#44] -(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] +(34) CometHashAggregate +Input [6]: [gross_margin#30, i_category#11, i_class#10, t_category#31, t_class#32, lochierarchy#33] +Keys [6]: [gross_margin#30, i_category#11, i_class#10, t_category#31, t_class#32, lochierarchy#33] Functions: [] -Aggregate Attributes: [] -Results [6]: [gross_margin#18, i_category#11, i_class#10, t_category#19, t_class#20, lochierarchy#21] -(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] +(35) CometColumnarExchange +Input [6]: [gross_margin#30, i_category#11, i_class#10, t_category#31, t_class#32, lochierarchy#33] +Arguments: hashpartitioning(gross_margin#30, i_category#11, i_class#10, t_category#31, t_class#32, lochierarchy#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(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] +(36) CometHashAggregate +Input [6]: [gross_margin#30, i_category#11, i_class#10, t_category#31, t_class#32, lochierarchy#33] +Keys [6]: [gross_margin#30, i_category#11, i_class#10, t_category#31, t_class#32, lochierarchy#33] Functions: [] -Aggregate Attributes: [] -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] -(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] +(37) CometColumnarExchange +Input [5]: [gross_margin#30, i_category#11, i_class#10, lochierarchy#33, _w0#45] +Arguments: hashpartitioning(lochierarchy#33, _w0#45, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(38) CometSort +Input [5]: [gross_margin#30, i_category#11, i_class#10, lochierarchy#33, _w0#45] +Arguments: [gross_margin#30, i_category#11, i_class#10, lochierarchy#33, _w0#45], [lochierarchy#33 ASC NULLS FIRST, _w0#45 ASC NULLS FIRST, gross_margin#30 ASC NULLS FIRST] -(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) ColumnarToRow [codegen id : 1] +Input [5]: [gross_margin#30, i_category#11, i_class#10, lochierarchy#33, _w0#45] (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] +Input [5]: [gross_margin#30, i_category#11, i_class#10, lochierarchy#33, _w0#45] +Arguments: [rank(gross_margin#30) windowspecdefinition(lochierarchy#33, _w0#45, gross_margin#30 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#46], [lochierarchy#33, _w0#45], [gross_margin#30 ASC NULLS FIRST] -(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) Project [codegen id : 2] +Output [5]: [gross_margin#30, i_category#11, i_class#10, lochierarchy#33, rank_within_parent#46] +Input [6]: [gross_margin#30, i_category#11, i_class#10, lochierarchy#33, _w0#45, rank_within_parent#46] (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] +Input [5]: [gross_margin#30, i_category#11, i_class#10, lochierarchy#33, rank_within_parent#46] +Arguments: 100, [lochierarchy#33 DESC NULLS LAST, CASE WHEN (lochierarchy#33 = 0) THEN i_category#11 END ASC NULLS FIRST, rank_within_parent#46 ASC NULLS FIRST], [gross_margin#30, i_category#11, i_class#10, lochierarchy#33, rank_within_parent#46] ===== Subqueries ===== 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 256fddfc8..9fc6f31f9 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,70 +1,54 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (12) + WholeStageCodegen (2) Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [gross_margin,lochierarchy,_w0] - WholeStageCodegen (11) - Sort [lochierarchy,_w0,gross_margin] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [lochierarchy,_w0] #1 - 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 (9) - HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - InputAdapter - Union - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_category,i_class,sum,sum,ss_net_profit,ss_ext_sales_price] - CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] - CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] - CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_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 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometFilter [i_item_sk,i_class,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #7 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - 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] #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 (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 #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 - ReusedExchange [i_category,i_class,sum,sum] #3 + CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] + CometColumnarExchange [lochierarchy,_w0] #1 + CometHashAggregate [gross_margin,i_category,i_class,lochierarchy,_w0,t_category,t_class] + CometColumnarExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 + CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] + CometUnion [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] + CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,sum,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange [i_category,i_class] #3 + CometHashAggregate [i_category,i_class,sum,sum,ss_net_profit,ss_ext_sales_price] + CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] + CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_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 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometFilter [i_item_sk,i_class,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometBroadcastExchange [s_store_sk] #7 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty,sum(ss_net_profit),sum(ss_ext_sales_price)] + CometColumnarExchange [i_category] #8 + CometHashAggregate [i_category,sum,isEmpty,sum,isEmpty,ss_net_profit,ss_ext_sales_price] + CometHashAggregate [ss_net_profit,ss_ext_sales_price,i_category,i_class,sum,sum,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] + ReusedExchange [i_category,i_class,sum,sum] #3 + CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty,sum(ss_net_profit),sum(ss_ext_sales_price)] + CometColumnarExchange #9 + CometHashAggregate [sum,isEmpty,sum,isEmpty,ss_net_profit,ss_ext_sales_price] + CometHashAggregate [ss_net_profit,ss_ext_sales_price,i_category,i_class,sum,sum,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] + ReusedExchange [i_category,i_class,sum,sum] #3 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 41d045790..b6ab28625 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 @@ -2,18 +2,18 @@ TakeOrderedAndProject (45) +- * Project (44) +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) + :- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) : :- * Project (28) : : +- * Filter (27) : : +- Window (26) : : +- * Filter (25) : : +- Window (24) - : : +- * Sort (23) - : : +- Exchange (22) - : : +- * HashAggregate (21) - : : +- Exchange (20) - : : +- * ColumnarToRow (19) + : : +- * ColumnarToRow (23) + : : +- CometSort (22) + : : +- CometColumnarExchange (21) + : : +- CometHashAggregate (20) + : : +- CometColumnarExchange (19) : : +- CometHashAggregate (18) : : +- CometProject (17) : : +- CometBroadcastHashJoin (16) @@ -32,18 +32,18 @@ TakeOrderedAndProject (45) : : +- CometBroadcastExchange (15) : : +- CometFilter (14) : : +- CometScan parquet spark_catalog.default.store (13) - : +- BroadcastExchange (35) - : +- * Project (34) - : +- Window (33) - : +- * Sort (32) - : +- Exchange (31) - : +- * HashAggregate (30) - : +- ReusedExchange (29) + : +- BroadcastExchange (34) + : +- * Project (33) + : +- Window (32) + : +- * ColumnarToRow (31) + : +- CometSort (30) + : +- ReusedExchange (29) +- BroadcastExchange (42) +- * Project (41) +- Window (40) - +- * Sort (39) - +- ReusedExchange (38) + +- * ColumnarToRow (39) + +- CometSort (38) + +- ReusedExchange (37) (1) Scan parquet spark_catalog.default.item @@ -135,120 +135,113 @@ Input [7]: [i_brand#2, i_category#3, ss_sales_price#6, d_year#10, d_moy#11, s_st 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))] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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#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 : 2] +(20) CometHashAggregate 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))#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#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] +(21) CometColumnarExchange +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, _w0#17] +Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(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 +(22) CometSort +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, _w0#17] +Arguments: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, _w0#17], [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] + +(23) ColumnarToRow [codegen id : 1] +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, _w0#17] (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#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] +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 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, 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#18], [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 : 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] +(25) Filter [codegen id : 2] +Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 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 [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 : 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 : 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#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum#27] - -(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#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 : 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#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 : 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#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 : 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)] +Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, _w0#17, rn#18] +Arguments: [avg(_w0#17) 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#19], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10] + +(27) Filter [codegen id : 7] +Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 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 : 7] +Output [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18] +Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, _w0#17, rn#18, avg_monthly_sales#19] + +(29) ReusedExchange [Reuses operator id: 21] +Output [7]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#16] + +(30) CometSort +Input [7]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#16] +Arguments: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#16], [i_category#20 ASC NULLS FIRST, i_brand#21 ASC NULLS FIRST, s_store_name#22 ASC NULLS FIRST, s_company_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] + +(31) ColumnarToRow [codegen id : 3] +Input [7]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#16] + +(32) Window +Input [7]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#16] +Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#20, i_brand#21, s_store_name#22, s_company_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#26], [i_category#20, i_brand#21, s_store_name#22, s_company_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] + +(33) Project [codegen id : 4] +Output [6]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, sum_sales#16 AS sum_sales#27, rn#26] +Input [8]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#16, rn#26] + +(34) BroadcastExchange +Input [6]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, sum_sales#27, rn#26] +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=3] + +(35) BroadcastHashJoin [codegen id : 7] +Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#18] +Right keys [5]: [i_category#20, i_brand#21, s_store_name#22, s_company_name#23, (rn#26 + 1)] Join type: Inner Join condition: None -(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] +(36) Project [codegen id : 7] +Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, sum_sales#27] +Input [15]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, i_category#20, i_brand#21, s_store_name#22, s_company_name#23, sum_sales#27, rn#26] + +(37) ReusedExchange [Reuses operator id: 21] +Output [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#16] -(38) ReusedExchange [Reuses operator id: 31] -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] +(38) CometSort +Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#16] +Arguments: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#16], [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_name#30 ASC NULLS FIRST, s_company_name#31 ASC NULLS FIRST, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] -(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 +(39) ColumnarToRow [codegen id : 5] +Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#16] (40) Window -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] +Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#16] +Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#34], [i_category#28, i_brand#29, s_store_name#30, s_company_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] -(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] +(41) Project [codegen id : 6] +Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#16 AS sum_sales#35, rn#34] +Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#16, rn#34] (42) BroadcastExchange -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] +Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#35, rn#34] +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] -(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)] +(43) BroadcastHashJoin [codegen id : 7] +Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#18] +Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#34 - 1)] Join type: Inner Join condition: None -(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] +(44) Project [codegen id : 7] +Output [7]: [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#19, sum_sales#16, sum_sales#27 AS psum#36, sum_sales#35 AS nsum#37] +Input [16]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, sum_sales#27, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#35, rn#34] (45) TakeOrderedAndProject -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] +Input [7]: [i_category#3, 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_moy#11 ASC NULLS FIRST], [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#19, sum_sales#16, psum#36, nsum#37] ===== Subqueries ===== @@ -275,6 +268,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=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] 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 eeeb5ba36..3c44ef74c 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 (13) + WholeStageCodegen (7) 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,67 +8,61 @@ 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 (4) + WholeStageCodegen (2) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (3) - Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum,ss_sales_price] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometFilter [i_item_sk,i_brand,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #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] + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 + CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum,sum(UnscaledValue(ss_sales_price))] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum,ss_sales_price] + CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_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_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #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 #7 - WholeStageCodegen (8) + WholeStageCodegen (4) 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 (7) - Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + WholeStageCodegen (3) + ColumnarToRow InputAdapter - 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 + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #1 InputAdapter - BroadcastExchange #9 - WholeStageCodegen (12) + BroadcastExchange #8 + WholeStageCodegen (6) 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 (11) - Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + WholeStageCodegen (5) + ColumnarToRow InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #1 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 07bd1d0ca..ed68082f6 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,81 +1,83 @@ == Physical Plan == -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) +* ColumnarToRow (79) ++- CometTakeOrderedAndProject (78) + +- CometHashAggregate (77) + +- CometColumnarExchange (76) + +- RowToColumnar (75) + +- * HashAggregate (74) + +- Union (73) + :- * Project (26) + : +- * Filter (25) + : +- Window (24) + : +- * Sort (23) + : +- Window (22) + : +- * ColumnarToRow (21) + : +- CometSort (20) + : +- CometColumnarExchange (19) + : +- CometHashAggregate (18) + : +- CometColumnarExchange (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) + : +- * ColumnarToRow (44) + : +- CometSort (43) + : +- CometColumnarExchange (42) + : +- CometHashAggregate (41) + : +- CometColumnarExchange (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) + +- * ColumnarToRow (67) + +- CometSort (66) + +- CometColumnarExchange (65) + +- CometHashAggregate (64) + +- CometColumnarExchange (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 @@ -155,312 +157,310 @@ Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, w 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))] -(17) ColumnarToRow [codegen id : 1] +(17) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(19) HashAggregate [codegen id : 2] +(18) CometHashAggregate 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))#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] -(20) Exchange -Input [3]: [item#26, return_ratio#27, currency_ratio#28] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] +(19) CometColumnarExchange +Input [3]: [item#22, return_ratio#23, currency_ratio#24] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(20) CometSort +Input [3]: [item#22, return_ratio#23, currency_ratio#24] +Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] -(21) Sort [codegen id : 3] -Input [3]: [item#26, return_ratio#27, currency_ratio#28] -Arguments: [return_ratio#27 ASC NULLS FIRST], false, 0 +(21) ColumnarToRow [codegen id : 1] +Input [3]: [item#22, return_ratio#23, currency_ratio#24] (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] +Input [3]: [item#22, return_ratio#23, currency_ratio#24] +Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] -(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 +(23) Sort [codegen id : 2] +Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] +Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 (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] +Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] +Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] -(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)) +(25) Filter [codegen id : 3] +Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] +Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) -(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] +(26) Project [codegen id : 3] +Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] (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] +Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#37), dynamicpruningexpression(cs_sold_date_sk#37 IN dynamicpruning#38)] +PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_sold_date_sk#33 IN dynamicpruning#34)] 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 (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)) +Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] +Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) (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] +Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] +Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] (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] +Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] +Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] (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] +Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] 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 (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)) +Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] +Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) (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] +Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] +Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] (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 +Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] +Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] +Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], 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] +Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] +Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] (36) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#44] +Output [1]: [d_date_sk#40] (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 +Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] +Right output [1]: [d_date_sk#40] +Arguments: [cs_sold_date_sk#33], [d_date_sk#40], 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] +Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] +Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] (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))] +Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] +Keys [1]: [cs_item_sk#28] +Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 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] +(40) CometColumnarExchange +Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] +Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(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] +(41) CometHashAggregate +Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] +Keys [1]: [cs_item_sk#28] +Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(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] +(42) CometColumnarExchange +Input [3]: [item#47, return_ratio#48, currency_ratio#49] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(43) Exchange -Input [3]: [item#55, return_ratio#56, currency_ratio#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +(43) CometSort +Input [3]: [item#47, return_ratio#48, currency_ratio#49] +Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] -(44) Sort [codegen id : 8] -Input [3]: [item#55, return_ratio#56, currency_ratio#57] -Arguments: [return_ratio#56 ASC NULLS FIRST], false, 0 +(44) ColumnarToRow [codegen id : 4] +Input [3]: [item#47, return_ratio#48, currency_ratio#49] (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] +Input [3]: [item#47, return_ratio#48, currency_ratio#49] +Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] -(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 +(46) Sort [codegen id : 5] +Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] +Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 (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] +Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] +Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] -(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)) +(48) Filter [codegen id : 6] +Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] +Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) -(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] +(49) Project [codegen id : 6] +Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] +Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] (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] +Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#66), dynamicpruningexpression(ss_sold_date_sk#66 IN dynamicpruning#67)] +PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] 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 (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)) +Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] +Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) (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] +Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] +Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] (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] +Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] +Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] (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] +Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] 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 (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)) +Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] +Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) (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] +Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] +Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] (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 +Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] +Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] +Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], 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] +Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] +Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] (59) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#73] +Output [1]: [d_date_sk#65] (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 +Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] +Right output [1]: [d_date_sk#65] +Arguments: [ss_sold_date_sk#58], [d_date_sk#65], 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] +Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] +Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] (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))] +Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] +Keys [1]: [ss_item_sk#53] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 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] +(63) CometColumnarExchange +Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] +Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(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] +(64) CometHashAggregate +Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] +Keys [1]: [ss_item_sk#53] +Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(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] +(65) CometColumnarExchange +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(66) Exchange -Input [3]: [item#84, return_ratio#85, currency_ratio#86] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +(66) CometSort +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] -(67) Sort [codegen id : 13] -Input [3]: [item#84, return_ratio#85, currency_ratio#86] -Arguments: [return_ratio#85 ASC NULLS FIRST], false, 0 +(67) ColumnarToRow [codegen id : 7] +Input [3]: [item#72, return_ratio#73, currency_ratio#74] (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] +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] -(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 +(69) Sort [codegen id : 8] +Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] +Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 (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] +Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] +Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] -(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)) +(71) Filter [codegen id : 9] +Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] +Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) -(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] +(72) Project [codegen id : 9] +Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] +Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] (73) Union -(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] +(74) HashAggregate [codegen id : 10] +Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(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] +(75) RowToColumnar +Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(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] +(76) CometColumnarExchange +Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(77) CometHashAggregate +Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] -(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] +(78) CometTakeOrderedAndProject +Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST,item#22 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST, item#22 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] + +(79) ColumnarToRow [codegen id : 11] +Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (82) -+- * ColumnarToRow (81) - +- CometProject (80) - +- CometFilter (79) - +- CometScan parquet spark_catalog.default.date_dim (78) +BroadcastExchange (84) ++- * ColumnarToRow (83) + +- CometProject (82) + +- CometFilter (81) + +- CometScan parquet spark_catalog.default.date_dim (80) -(78) Scan parquet spark_catalog.default.date_dim +(80) 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 -(79) CometFilter +(81) 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)) -(80) CometProject +(82) CometProject Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(81) ColumnarToRow [codegen id : 1] +(83) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(82) BroadcastExchange +(84) 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 = 27 Hosting Expression = cs_sold_date_sk#37 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#66 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 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 0e6b65b06..0d02a9c39 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,121 +1,111 @@ -TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] - WholeStageCodegen (17) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Exchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (16) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (5) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (4) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (3) - Sort [return_ratio] - InputAdapter - Exchange #2 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,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_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - 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 (9) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (8) - Sort [return_ratio] - InputAdapter - 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] #8 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometHashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,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_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_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] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - 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 (14) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (13) - Sort [return_ratio] - InputAdapter - 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] #11 - WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometHashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,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_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_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] - ReusedSubquery [d_date_sk] #1 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - ReusedExchange [d_date_sk] #6 +WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] + CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] + CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] + InputAdapter + Union + WholeStageCodegen (3) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (2) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #2 + CometHashAggregate [item,return_ratio,currency_ratio,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))] + CometColumnarExchange [ws_item_sk] #3 + CometHashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,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_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + WholeStageCodegen (6) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (5) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #7 + CometHashAggregate [item,return_ratio,currency_ratio,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))] + CometColumnarExchange [cs_item_sk] #8 + CometHashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,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_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_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] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (9) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (8) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #10 + CometHashAggregate [item,return_ratio,currency_ratio,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))] + CometColumnarExchange [ss_item_sk] #11 + CometHashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,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_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_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] + ReusedSubquery [d_date_sk] #1 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + 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 1bcaa1ea1..20a3f1c52 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,74 +1,84 @@ == Physical Plan == -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) +TakeOrderedAndProject (80) ++- * Filter (79) + +- * HashAggregate (78) + +- * HashAggregate (77) + +- * Project (76) + +- * BroadcastHashJoin Inner BuildRight (75) + :- Window (68) + : +- * ColumnarToRow (67) + : +- CometSort (66) + : +- CometColumnarExchange (65) + : +- CometProject (64) + : +- CometFilter (63) + : +- CometSortMergeJoin (62) + : :- CometSort (32) + : : +- CometColumnarExchange (31) + : : +- RowToColumnar (30) + : : +- * HashAggregate (29) + : : +- * ColumnarToRow (28) + : : +- CometColumnarExchange (27) + : : +- RowToColumnar (26) + : : +- * HashAggregate (25) + : : +- * Project (24) + : : +- * BroadcastHashJoin Inner BuildRight (23) + : : :- * Project (16) + : : : +- Window (15) + : : : +- * ColumnarToRow (14) + : : : +- CometSort (13) + : : : +- CometColumnarExchange (12) + : : : +- CometHashAggregate (11) + : : : +- CometColumnarExchange (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 (22) + : : +- * Project (21) + : : +- Window (20) + : : +- * ColumnarToRow (19) + : : +- CometSort (18) + : : +- ReusedExchange (17) + : +- CometSort (61) + : +- CometColumnarExchange (60) + : +- RowToColumnar (59) + : +- * HashAggregate (58) + : +- * ColumnarToRow (57) + : +- CometColumnarExchange (56) + : +- RowToColumnar (55) + : +- * HashAggregate (54) + : +- * Project (53) + : +- * BroadcastHashJoin Inner BuildRight (52) + : :- * Project (45) + : : +- Window (44) + : : +- * ColumnarToRow (43) + : : +- CometSort (42) + : : +- CometColumnarExchange (41) + : : +- CometHashAggregate (40) + : : +- CometColumnarExchange (39) + : : +- CometHashAggregate (38) + : : +- CometProject (37) + : : +- CometBroadcastHashJoin (36) + : : :- CometFilter (34) + : : : +- CometScan parquet spark_catalog.default.store_sales (33) + : : +- ReusedExchange (35) + : +- BroadcastExchange (51) + : +- * Project (50) + : +- Window (49) + : +- * ColumnarToRow (48) + : +- CometSort (47) + : +- ReusedExchange (46) + +- BroadcastExchange (74) + +- * Project (73) + +- Window (72) + +- * ColumnarToRow (71) + +- CometSort (70) + +- ReusedExchange (69) (1) Scan parquet spark_catalog.default.web_sales @@ -116,314 +126,339 @@ 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))] -(10) ColumnarToRow [codegen id : 1] +(10) CometColumnarExchange Input [3]: [ws_item_sk#1, d_date#6, sum#8] +Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(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] - -(12) HashAggregate [codegen id : 2] +(11) CometHashAggregate 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] -(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] +(12) CometColumnarExchange +Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(13) CometSort +Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] +Arguments: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(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 +(14) ColumnarToRow [codegen id : 1] +Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] (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] +Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] +Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(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] +(16) Project [codegen id : 4] +Output [4]: [item_sk#9, d_date#6, sumws#10, rk#11] +Input [5]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1, rk#11] -(17) ReusedExchange [Reuses operator id: 13] -Output [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] +(17) ReusedExchange [Reuses operator id: 12] +Output [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] -(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 +(18) CometSort +Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] +Arguments: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13], [ws_item_sk#13 ASC NULLS FIRST, d_date#12 ASC NULLS FIRST] -(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] +(19) ColumnarToRow [codegen id : 2] +Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] -(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] +(20) Window +Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] +Arguments: [row_number() windowspecdefinition(ws_item_sk#13, d_date#12 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#14], [ws_item_sk#13], [d_date#12 ASC NULLS FIRST] -(21) BroadcastExchange -Input [3]: [item_sk#16, sumws#17, rk#15] +(21) Project [codegen id : 3] +Output [3]: [item_sk#9 AS item_sk#15, sumws#10 AS sumws#16, rk#14] +Input [5]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13, rk#14] + +(22) BroadcastExchange +Input [3]: [item_sk#15, sumws#16, rk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(22) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [item_sk#10] -Right keys [1]: [item_sk#16] +(23) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [item_sk#9] +Right keys [1]: [item_sk#15] Join type: Inner -Join condition: (rk#12 >= rk#15) - -(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] - -(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] - -(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] - -(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] - -(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] - -(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 - -(29) 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#11 >= rk#14) + +(24) Project [codegen id : 4] +Output [4]: [item_sk#9, d_date#6, sumws#10, sumws#16] +Input [7]: [item_sk#9, d_date#6, sumws#10, rk#11, item_sk#15, sumws#16, rk#14] + +(25) HashAggregate [codegen id : 4] +Input [4]: [item_sk#9, d_date#6, sumws#10, sumws#16] +Keys [3]: [item_sk#9, d_date#6, sumws#10] +Functions [1]: [partial_sum(sumws#16)] +Aggregate Attributes [2]: [sum#17, isEmpty#18] +Results [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] + +(26) RowToColumnar +Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] + +(27) CometColumnarExchange +Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] +Arguments: hashpartitioning(item_sk#9, d_date#6, sumws#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(28) ColumnarToRow [codegen id : 5] +Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] + +(29) HashAggregate [codegen id : 5] +Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] +Keys [3]: [item_sk#9, d_date#6, sumws#10] +Functions [1]: [sum(sumws#16)] +Aggregate Attributes [1]: [sum(sumws#16)#21] +Results [3]: [item_sk#9, d_date#6, sum(sumws#16)#21 AS cume_sales#22] + +(30) RowToColumnar +Input [3]: [item_sk#9, d_date#6, cume_sales#22] + +(31) CometColumnarExchange +Input [3]: [item_sk#9, d_date#6, cume_sales#22] +Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(32) CometSort +Input [3]: [item_sk#9, d_date#6, cume_sales#22] +Arguments: [item_sk#9, d_date#6, cume_sales#22], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] + +(33) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] 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#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(30) CometFilter -Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_item_sk#24) +(34) CometFilter +Input [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] +Condition : isnotnull(ss_item_sk#23) + +(35) ReusedExchange [Reuses operator id: 6] +Output [2]: [d_date_sk#27, d_date#28] -(31) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#28, d_date#29] +(36) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] +Right output [2]: [d_date_sk#27, d_date#28] +Arguments: [ss_sold_date_sk#25], [d_date_sk#27], Inner, BuildRight -(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 +(37) CometProject +Input [5]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25, d_date_sk#27, d_date#28] +Arguments: [ss_item_sk#23, ss_sales_price#24, d_date#28], [ss_item_sk#23, ss_sales_price#24, d_date#28] -(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] +(38) CometHashAggregate +Input [3]: [ss_item_sk#23, ss_sales_price#24, d_date#28] +Keys [2]: [ss_item_sk#23, d_date#28] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#24))] -(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))] +(39) CometColumnarExchange +Input [3]: [ss_item_sk#23, d_date#28, sum#29] +Arguments: hashpartitioning(ss_item_sk#23, d_date#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(35) ColumnarToRow [codegen id : 11] -Input [3]: [ss_item_sk#24, d_date#29, sum#30] +(40) CometHashAggregate +Input [3]: [ss_item_sk#23, d_date#28, sum#29] +Keys [2]: [ss_item_sk#23, d_date#28] +Functions [1]: [sum(UnscaledValue(ss_sales_price#24))] -(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] +(41) CometColumnarExchange +Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] +Arguments: hashpartitioning(ss_item_sk#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(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))#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] +(42) CometSort +Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] +Arguments: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23], [ss_item_sk#23 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] -(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] +(43) ColumnarToRow [codegen id : 6] +Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -(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 +(44) Window +Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] +Arguments: [row_number() windowspecdefinition(ss_item_sk#23, d_date#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#32], [ss_item_sk#23], [d_date#28 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] +(45) Project [codegen id : 9] +Output [4]: [item_sk#30, d_date#28, sumss#31, rk#32] +Input [5]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23, rk#32] -(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] +(46) ReusedExchange [Reuses operator id: 41] +Output [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] -(42) ReusedExchange [Reuses operator id: 38] -Output [4]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36] +(47) CometSort +Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] +Arguments: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34], [ss_item_sk#34 ASC NULLS FIRST, d_date#33 ASC NULLS FIRST] -(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 +(48) ColumnarToRow [codegen id : 7] +Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] -(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] +(49) Window +Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] +Arguments: [row_number() windowspecdefinition(ss_item_sk#34, d_date#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#35], [ss_item_sk#34], [d_date#33 ASC NULLS FIRST] -(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] +(50) Project [codegen id : 8] +Output [3]: [item_sk#30 AS item_sk#36, sumss#31 AS sumss#37, rk#35] +Input [5]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34, rk#35] -(46) BroadcastExchange -Input [3]: [item_sk#38, sumss#39, rk#37] +(51) BroadcastExchange +Input [3]: [item_sk#36, sumss#37, rk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -(47) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_sk#32] -Right keys [1]: [item_sk#38] +(52) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [item_sk#30] +Right keys [1]: [item_sk#36] Join type: Inner -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#32, d_date#29] -Join type: FullOuter -Join condition: None - -(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) - -(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] - -(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] - -(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 - -(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] - -(60) ReusedExchange [Reuses operator id: 57] -Output [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] - -(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 - -(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] - -(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] - -(64) BroadcastExchange -Input [4]: [item_sk#52, web_sales#53, store_sales#54, rk#51] +Join condition: (rk#32 >= rk#35) + +(53) Project [codegen id : 9] +Output [4]: [item_sk#30, d_date#28, sumss#31, sumss#37] +Input [7]: [item_sk#30, d_date#28, sumss#31, rk#32, item_sk#36, sumss#37, rk#35] + +(54) HashAggregate [codegen id : 9] +Input [4]: [item_sk#30, d_date#28, sumss#31, sumss#37] +Keys [3]: [item_sk#30, d_date#28, sumss#31] +Functions [1]: [partial_sum(sumss#37)] +Aggregate Attributes [2]: [sum#38, isEmpty#39] +Results [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] + +(55) RowToColumnar +Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] + +(56) CometColumnarExchange +Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] +Arguments: hashpartitioning(item_sk#30, d_date#28, sumss#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(57) ColumnarToRow [codegen id : 10] +Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] + +(58) HashAggregate [codegen id : 10] +Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] +Keys [3]: [item_sk#30, d_date#28, sumss#31] +Functions [1]: [sum(sumss#37)] +Aggregate Attributes [1]: [sum(sumss#37)#42] +Results [3]: [item_sk#30, d_date#28, sum(sumss#37)#42 AS cume_sales#43] + +(59) RowToColumnar +Input [3]: [item_sk#30, d_date#28, cume_sales#43] + +(60) CometColumnarExchange +Input [3]: [item_sk#30, d_date#28, cume_sales#43] +Arguments: hashpartitioning(item_sk#30, d_date#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(61) CometSort +Input [3]: [item_sk#30, d_date#28, cume_sales#43] +Arguments: [item_sk#30, d_date#28, cume_sales#43], [item_sk#30 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] + +(62) CometSortMergeJoin +Left output [3]: [item_sk#9, d_date#6, cume_sales#22] +Right output [3]: [item_sk#30, d_date#28, cume_sales#43] +Arguments: [item_sk#9, d_date#6], [item_sk#30, d_date#28], FullOuter + +(63) CometFilter +Input [6]: [item_sk#9, d_date#6, cume_sales#22, item_sk#30, d_date#28, cume_sales#43] +Condition : isnotnull(CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END) + +(64) CometProject +Input [6]: [item_sk#9, d_date#6, cume_sales#22, item_sk#30, d_date#28, cume_sales#43] +Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END AS item_sk#44, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#28 END AS d_date#45, cume_sales#22 AS web_sales#46, cume_sales#43 AS store_sales#47] + +(65) CometColumnarExchange +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Arguments: hashpartitioning(item_sk#44, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] + +(66) CometSort +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST] + +(67) ColumnarToRow [codegen id : 11] +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] + +(68) Window +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Arguments: [row_number() windowspecdefinition(item_sk#44, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#48], [item_sk#44], [d_date#45 ASC NULLS FIRST] + +(69) ReusedExchange [Reuses operator id: 65] +Output [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] + +(70) CometSort +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST] + +(71) ColumnarToRow [codegen id : 22] +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] + +(72) Window +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Arguments: [row_number() windowspecdefinition(item_sk#44, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#49], [item_sk#44], [d_date#45 ASC NULLS FIRST] + +(73) Project [codegen id : 23] +Output [4]: [item_sk#44 AS item_sk#50, web_sales#46 AS web_sales#51, store_sales#47 AS store_sales#52, rk#49] +Input [5]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, rk#49] + +(74) BroadcastExchange +Input [4]: [item_sk#50, web_sales#51, store_sales#52, rk#49] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -(65) BroadcastHashJoin [codegen id : 46] -Left keys [1]: [item_sk#46] -Right keys [1]: [item_sk#52] +(75) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [item_sk#44] +Right keys [1]: [item_sk#50] Join type: Inner -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] +Join condition: (rk#48 >= rk#49) + +(76) Project [codegen id : 24] +Output [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_sales#51, store_sales#52] +Input [9]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, rk#48, item_sk#50, web_sales#51, store_sales#52, rk#49] + +(77) HashAggregate [codegen id : 24] +Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_sales#51, store_sales#52] +Keys [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Functions [2]: [partial_max(web_sales#51), partial_max(store_sales#52)] +Aggregate Attributes [2]: [max#53, max#54] +Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#55, max#56] + +(78) HashAggregate [codegen id : 24] +Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#55, max#56] +Keys [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Functions [2]: [max(web_sales#51), max(store_sales#52)] +Aggregate Attributes [2]: [max(web_sales#51)#57, max(store_sales#52)#58] +Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max(web_sales#51)#57 AS web_cumulative#59, max(store_sales#52)#58 AS store_cumulative#60] + +(79) Filter [codegen id : 24] +Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] +Condition : ((isnotnull(web_cumulative#59) AND isnotnull(store_cumulative#60)) AND (web_cumulative#59 > store_cumulative#60)) + +(80) TakeOrderedAndProject +Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] +Arguments: 100, [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (75) -+- * ColumnarToRow (74) - +- CometProject (73) - +- CometFilter (72) - +- CometScan parquet spark_catalog.default.date_dim (71) +BroadcastExchange (85) ++- * ColumnarToRow (84) + +- CometProject (83) + +- CometFilter (82) + +- CometScan parquet spark_catalog.default.date_dim (81) -(71) Scan parquet spark_catalog.default.date_dim +(81) 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 -(72) CometFilter +(82) 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)) -(73) CometProject +(83) 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] -(74) ColumnarToRow [codegen id : 1] +(84) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(75) BroadcastExchange +(85) 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 = 29 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#25 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 e9c4d46ca..7fe01fcc8 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 (46) + WholeStageCodegen (24) 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,119 +7,114 @@ 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 (22) - Sort [item_sk,d_date] + WholeStageCodegen (11) + ColumnarToRow InputAdapter - Exchange [item_sk] #1 - 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 (10) - Sort [item_sk,d_date] - InputAdapter - Exchange [item_sk,d_date] #2 - 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 (8) + CometSort [item_sk,d_date,web_sales,store_sales] + CometColumnarExchange [item_sk] #1 + CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] + CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #2 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [item_sk,d_date,sumws] #3 + RowToColumnar + WholeStageCodegen (4) 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 (3) - Sort [ws_item_sk,d_date] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [ws_item_sk] #4 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [ws_item_sk,d_date,sum,ws_sales_price] - CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometSort [item_sk,d_date,sumws,ws_item_sk] + CometColumnarExchange [ws_item_sk] #4 + CometHashAggregate [item_sk,d_date,sumws,ws_item_sk,sum,sum(UnscaledValue(ws_sales_price))] + CometColumnarExchange [ws_item_sk,d_date] #5 + CometHashAggregate [ws_item_sk,d_date,sum,ws_sales_price] + CometProject [ws_item_sk,ws_sales_price,d_date] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange [d_date_sk,d_date] #7 + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter BroadcastExchange #8 - WholeStageCodegen (7) + WholeStageCodegen (3) Project [item_sk,sumws,rk] InputAdapter Window [ws_item_sk,d_date] - WholeStageCodegen (6) - Sort [ws_item_sk,d_date] + WholeStageCodegen (2) + ColumnarToRow InputAdapter - ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 - InputAdapter - WholeStageCodegen (20) - Sort [item_sk,d_date] - InputAdapter - 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] #10 - WholeStageCodegen (18) + CometSort [item_sk,d_date,sumws,ws_item_sk] + ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #9 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [item_sk,d_date,sumss] #10 + RowToColumnar + WholeStageCodegen (9) 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 (13) - Sort [ss_item_sk,d_date] + WholeStageCodegen (6) + ColumnarToRow InputAdapter - 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] #12 - WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometHashAggregate [ss_item_sk,d_date,sum,ss_sales_price] - CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_date] #7 + CometSort [item_sk,d_date,sumss,ss_item_sk] + CometColumnarExchange [ss_item_sk] #11 + CometHashAggregate [item_sk,d_date,sumss,ss_item_sk,sum,sum(UnscaledValue(ss_sales_price))] + CometColumnarExchange [ss_item_sk,d_date] #12 + CometHashAggregate [ss_item_sk,d_date,sum,ss_sales_price] + CometProject [ss_item_sk,ss_sales_price,d_date] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_date] #7 InputAdapter BroadcastExchange #13 - WholeStageCodegen (17) + WholeStageCodegen (8) Project [item_sk,sumss,rk] InputAdapter Window [ss_item_sk,d_date] - WholeStageCodegen (16) - Sort [ss_item_sk,d_date] + WholeStageCodegen (7) + ColumnarToRow InputAdapter - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 + CometSort [item_sk,d_date,sumss,ss_item_sk] + ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 InputAdapter BroadcastExchange #14 - WholeStageCodegen (45) + WholeStageCodegen (23) Project [item_sk,web_sales,store_sales,rk] InputAdapter Window [item_sk,d_date] - WholeStageCodegen (44) - Sort [item_sk,d_date] + WholeStageCodegen (22) + ColumnarToRow InputAdapter - ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 + CometSort [item_sk,d_date,web_sales,store_sales] + 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 1b66eb4da..59b4666bf 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 @@ -2,18 +2,18 @@ TakeOrderedAndProject (45) +- * Project (44) +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) + :- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) : :- * Project (28) : : +- * Filter (27) : : +- Window (26) : : +- * Filter (25) : : +- Window (24) - : : +- * Sort (23) - : : +- Exchange (22) - : : +- * HashAggregate (21) - : : +- Exchange (20) - : : +- * ColumnarToRow (19) + : : +- * ColumnarToRow (23) + : : +- CometSort (22) + : : +- CometColumnarExchange (21) + : : +- CometHashAggregate (20) + : : +- CometColumnarExchange (19) : : +- CometHashAggregate (18) : : +- CometProject (17) : : +- CometBroadcastHashJoin (16) @@ -32,18 +32,18 @@ TakeOrderedAndProject (45) : : +- CometBroadcastExchange (15) : : +- CometFilter (14) : : +- CometScan parquet spark_catalog.default.call_center (13) - : +- BroadcastExchange (35) - : +- * Project (34) - : +- Window (33) - : +- * Sort (32) - : +- Exchange (31) - : +- * HashAggregate (30) - : +- ReusedExchange (29) + : +- BroadcastExchange (34) + : +- * Project (33) + : +- Window (32) + : +- * ColumnarToRow (31) + : +- CometSort (30) + : +- ReusedExchange (29) +- BroadcastExchange (42) +- * Project (41) +- Window (40) - +- * Sort (39) - +- ReusedExchange (38) + +- * ColumnarToRow (39) + +- CometSort (38) + +- ReusedExchange (37) (1) Scan parquet spark_catalog.default.item @@ -135,120 +135,113 @@ Input [6]: [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_n 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))] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(20) Exchange -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 : 2] +(20) CometHashAggregate 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))#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#16, _w0#17] -Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(21) CometColumnarExchange +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16] +Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(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 +(22) CometSort +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16] +Arguments: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16], [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] + +(23) ColumnarToRow [codegen id : 1] +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16] (24) Window -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] +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16] +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#17], [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 : 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] +(25) Filter [codegen id : 2] +Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16, rn#17] 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#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 : 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 : 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#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum#25] - -(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#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 : 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#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 : 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#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 : 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)] +Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16, rn#17] +Arguments: [avg(_w0#16) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#18], [i_category#3, i_brand#2, cc_name#13, d_year#10] + +(27) Filter [codegen id : 7] +Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16, rn#17, avg_monthly_sales#18] +Condition : ((isnotnull(avg_monthly_sales#18) AND (avg_monthly_sales#18 > 0.000000)) AND CASE WHEN (avg_monthly_sales#18 > 0.000000) THEN ((abs((sum_sales#15 - avg_monthly_sales#18)) / avg_monthly_sales#18) > 0.1000000000000000) END) + +(28) Project [codegen id : 7] +Output [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, avg_monthly_sales#18, rn#17] +Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, _w0#16, rn#17, avg_monthly_sales#18] + +(29) ReusedExchange [Reuses operator id: 21] +Output [6]: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15] + +(30) CometSort +Input [6]: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15] +Arguments: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15], [i_category#19 ASC NULLS FIRST, i_brand#20 ASC NULLS FIRST, cc_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] + +(31) ColumnarToRow [codegen id : 3] +Input [6]: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15] + +(32) Window +Input [6]: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15] +Arguments: [rank(d_year#22, d_moy#23) windowspecdefinition(i_category#19, i_brand#20, cc_name#21, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#24], [i_category#19, i_brand#20, cc_name#21], [d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] + +(33) Project [codegen id : 4] +Output [5]: [i_category#19, i_brand#20, cc_name#21, sum_sales#15 AS sum_sales#25, rn#24] +Input [7]: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15, rn#24] + +(34) BroadcastExchange +Input [5]: [i_category#19, i_brand#20, cc_name#21, sum_sales#25, rn#24] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=3] + +(35) BroadcastHashJoin [codegen id : 7] +Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#17] +Right keys [4]: [i_category#19, i_brand#20, cc_name#21, (rn#24 + 1)] Join type: Inner Join condition: None -(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] +(36) Project [codegen id : 7] +Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, avg_monthly_sales#18, rn#17, sum_sales#25] +Input [13]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, avg_monthly_sales#18, rn#17, i_category#19, i_brand#20, cc_name#21, sum_sales#25, rn#24] + +(37) ReusedExchange [Reuses operator id: 21] +Output [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#15] -(38) ReusedExchange [Reuses operator id: 31] -Output [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#16] +(38) CometSort +Input [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#15] +Arguments: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#15], [i_category#26 ASC NULLS FIRST, i_brand#27 ASC NULLS FIRST, cc_name#28 ASC NULLS FIRST, d_year#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST] -(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 +(39) ColumnarToRow [codegen id : 5] +Input [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#15] (40) Window -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] +Input [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#15] +Arguments: [rank(d_year#29, d_moy#30) windowspecdefinition(i_category#26, i_brand#27, cc_name#28, d_year#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#31], [i_category#26, i_brand#27, cc_name#28], [d_year#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST] -(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] +(41) Project [codegen id : 6] +Output [5]: [i_category#26, i_brand#27, cc_name#28, sum_sales#15 AS sum_sales#32, rn#31] +Input [7]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#15, rn#31] (42) BroadcastExchange -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] +Input [5]: [i_category#26, i_brand#27, cc_name#28, sum_sales#32, rn#31] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=4] -(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)] +(43) BroadcastHashJoin [codegen id : 7] +Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#17] +Right keys [4]: [i_category#26, i_brand#27, cc_name#28, (rn#31 - 1)] Join type: Inner Join condition: None -(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] +(44) Project [codegen id : 7] +Output [8]: [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#18, sum_sales#15, sum_sales#25 AS psum#33, sum_sales#32 AS nsum#34] +Input [14]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#15, avg_monthly_sales#18, rn#17, sum_sales#25, i_category#26, i_brand#27, cc_name#28, sum_sales#32, rn#31] (45) TakeOrderedAndProject -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] +Input [8]: [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#18, sum_sales#15, psum#33, nsum#34] +Arguments: 100, [(sum_sales#15 - avg_monthly_sales#18) ASC NULLS FIRST, d_year#10 ASC NULLS FIRST], [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#18, sum_sales#15, psum#33, nsum#34] ===== Subqueries ===== @@ -275,6 +268,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=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] 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 968f7e1af..ed9e697f3 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 (13) + WholeStageCodegen (7) 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,67 +8,61 @@ 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 (4) + WholeStageCodegen (2) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (3) - Sort [i_category,i_brand,cc_name,d_year,d_moy] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [i_category,i_brand,cc_name] #1 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum,cs_sales_price] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometFilter [i_item_sk,i_brand,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 - CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_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_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,cc_name] #1 + CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum,sum(UnscaledValue(cs_sales_price))] + CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum,cs_sales_price] + CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_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_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 + CometFilter [cc_call_center_sk,cc_name] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter BroadcastExchange #7 - WholeStageCodegen (8) + WholeStageCodegen (4) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (7) - Sort [i_category,i_brand,cc_name,d_year,d_moy] + WholeStageCodegen (3) + ColumnarToRow InputAdapter - 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 + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #1 InputAdapter - BroadcastExchange #9 - WholeStageCodegen (12) + BroadcastExchange #8 + WholeStageCodegen (6) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (11) - Sort [i_category,i_brand,cc_name,d_year,d_moy] + WholeStageCodegen (5) + ColumnarToRow InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #1 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 3ca1d3551..bb6a6a8f4 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 @@ -1,87 +1,97 @@ == Physical Plan == -TakeOrderedAndProject (83) -+- * HashAggregate (82) - +- Exchange (81) - +- * HashAggregate (80) - +- Union (79) - :- * HashAggregate (68) - : +- Exchange (67) - : +- * HashAggregate (66) - : +- Union (65) - : :- * 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) - : +- * 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) - : +- Exchange (72) - : +- * HashAggregate (71) - : +- * HashAggregate (70) - : +- ReusedExchange (69) - +- * HashAggregate (78) - +- Exchange (77) - +- * HashAggregate (76) - +- * HashAggregate (75) - +- ReusedExchange (74) +* ColumnarToRow (93) ++- CometTakeOrderedAndProject (92) + +- CometHashAggregate (91) + +- CometColumnarExchange (90) + +- RowToColumnar (89) + +- * HashAggregate (88) + +- Union (87) + :- * HashAggregate (70) + : +- * ColumnarToRow (69) + : +- CometColumnarExchange (68) + : +- RowToColumnar (67) + : +- * HashAggregate (66) + : +- Union (65) + : :- * HashAggregate (22) + : : +- * ColumnarToRow (21) + : : +- CometColumnarExchange (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) + : : +- * ColumnarToRow (40) + : : +- CometColumnarExchange (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) + : +- * ColumnarToRow (63) + : +- CometColumnarExchange (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 (78) + : +- * ColumnarToRow (77) + : +- CometColumnarExchange (76) + : +- RowToColumnar (75) + : +- * HashAggregate (74) + : +- * HashAggregate (73) + : +- * ColumnarToRow (72) + : +- ReusedExchange (71) + +- * HashAggregate (86) + +- * ColumnarToRow (85) + +- CometColumnarExchange (84) + +- RowToColumnar (83) + +- * HashAggregate (82) + +- * HashAggregate (81) + +- * ColumnarToRow (80) + +- ReusedExchange (79) (1) Scan parquet spark_catalog.default.store_sales @@ -177,14 +187,14 @@ 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))] -(20) ColumnarToRow [codegen id : 1] +(20) CometColumnarExchange Input [5]: [s_store_id#25, sum#26, sum#27, sum#28, sum#29] +Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(21) Exchange +(21) ColumnarToRow [codegen id : 1] 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] -(22) HashAggregate [codegen id : 2] +(22) HashAggregate [codegen id : 1] 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))] @@ -268,14 +278,14 @@ Input [5]: [sales_price#46, profit#47, return_amt#48, net_loss#49, cp_catalog_pa 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] +(39) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=2] -(40) Exchange +(40) ColumnarToRow [codegen id : 2] 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] +(41) HashAggregate [codegen id : 2] 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))] @@ -378,14 +388,14 @@ Input [5]: [sales_price#83, profit#84, return_amt#85, net_loss#86, web_site_id#1 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) ColumnarToRow [codegen id : 5] +(62) CometColumnarExchange Input [5]: [web_site_id#104, sum#105, sum#106, sum#107, sum#108] +Arguments: hashpartitioning(web_site_id#104, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(63) Exchange +(63) ColumnarToRow [codegen id : 3] 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 : 6] +(64) HashAggregate [codegen id : 3] 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))] @@ -394,133 +404,161 @@ Results [5]: [web channel AS channel#113, concat(web_site, web_site_id#104) AS i (65) Union -(66) HashAggregate [codegen id : 7] +(66) HashAggregate [codegen id : 4] 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 +(67) RowToColumnar 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 : 8] +(68) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=4] + +(69) ColumnarToRow [codegen id : 5] +Input [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] + +(70) HashAggregate [codegen id : 5] 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] +(71) ReusedExchange [Reuses operator id: 68] Output [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] -(70) HashAggregate [codegen id : 16] +(72) ColumnarToRow [codegen id : 10] +Input [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] + +(73) HashAggregate [codegen id : 10] 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] +(74) HashAggregate [codegen id : 10] 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 +(75) RowToColumnar 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 : 17] +(76) CometColumnarExchange +Input [7]: [channel#34, sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150] +Arguments: hashpartitioning(channel#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(77) ColumnarToRow [codegen id : 11] +Input [7]: [channel#34, sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150] + +(78) HashAggregate [codegen id : 11] 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] +(79) ReusedExchange [Reuses operator id: 68] Output [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] -(75) HashAggregate [codegen id : 25] +(80) ColumnarToRow [codegen id : 16] +Input [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] + +(81) 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 [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 : 25] +(82) HashAggregate [codegen id : 16] Input [3]: [sales#136, returns#137, profit#138] Keys: [] 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 +(83) RowToColumnar +Input [6]: [sum#164, isEmpty#165, sum#166, isEmpty#167, sum#168, isEmpty#169] + +(84) CometColumnarExchange Input [6]: [sum#164, isEmpty#165, sum#166, isEmpty#167, sum#168, isEmpty#169] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(78) HashAggregate [codegen id : 26] +(85) ColumnarToRow [codegen id : 17] +Input [6]: [sum#164, isEmpty#165, sum#166, isEmpty#167, sum#168, isEmpty#169] + +(86) HashAggregate [codegen id : 17] Input [6]: [sum#164, isEmpty#165, sum#166, isEmpty#167, sum#168, isEmpty#169] Keys: [] 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 +(87) Union -(80) HashAggregate [codegen id : 27] +(88) HashAggregate [codegen id : 18] 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#34, id#35, sales#133, returns#134, profit#135] -(81) Exchange +(89) RowToColumnar 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 : 28] +(90) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=7] + +(91) CometHashAggregate 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#34, id#35, sales#133, returns#134, profit#135] -(83) TakeOrderedAndProject +(92) CometTakeOrderedAndProject +Input [5]: [channel#34, id#35, sales#133, returns#134, profit#135] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#34 ASC NULLS FIRST,id#35 ASC NULLS FIRST], output=[channel#34,id#35,sales#133,returns#134,profit#135]), [channel#34, id#35, sales#133, returns#134, profit#135], 100, [channel#34 ASC NULLS FIRST, id#35 ASC NULLS FIRST], [channel#34, id#35, sales#133, returns#134, profit#135] + +(93) ColumnarToRow [codegen id : 19] 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 ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (88) -+- * ColumnarToRow (87) - +- CometProject (86) - +- CometFilter (85) - +- CometScan parquet spark_catalog.default.date_dim (84) +BroadcastExchange (98) ++- * ColumnarToRow (97) + +- CometProject (96) + +- CometFilter (95) + +- CometScan parquet spark_catalog.default.date_dim (94) -(84) Scan parquet spark_catalog.default.date_dim +(94) 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 -(85) CometFilter +(95) 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)) -(86) CometProject +(96) CometProject Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(87) ColumnarToRow [codegen id : 1] +(97) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(88) BroadcastExchange +(98) BroadcastExchange Input [1]: [d_date_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] 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 1428c7ba5..00dd3c480 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,124 +1,128 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (28) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Exchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (27) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - 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 (7) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] - CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] - CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - 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,ss_ext_sales_price,ss_net_profit,ss_sold_date_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 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_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 - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #6 - CometFilter [s_store_sk,s_store_id] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] +WholeStageCodegen (19) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [channel,id,sales,returns,profit] + CometColumnarExchange [channel,id,sales,returns,profit] #1 + RowToColumnar + WholeStageCodegen (18) + HashAggregate [channel,id,sales,returns,profit] + InputAdapter + Union + WholeStageCodegen (5) + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel,id] #2 + RowToColumnar 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] + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange [cp_catalog_page_id] #7 + Union + WholeStageCodegen (1) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [s_store_id] #3 + CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] + CometProject [sales_price,profit,return_amt,net_loss,s_store_id] + CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] + CometProject [store_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + 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,ss_ext_sales_price,ss_net_profit,ss_sold_date_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 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_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 + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [s_store_sk,s_store_id] #6 + CometFilter [s_store_sk,s_store_id] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + WholeStageCodegen (2) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [cp_catalog_page_id] #7 + CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] + CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [page_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + 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,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_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 + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #8 + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] - CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - 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,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_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 - ReusedExchange [d_date_sk] #5 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #8 - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - 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] #9 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] - CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] - CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - 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,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - 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,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] - CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #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_web_site_sk,ws_order_number,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - ReusedExchange [d_date_sk] #5 - CometBroadcastExchange [web_site_sk,web_site_id] #11 - CometFilter [web_site_sk,web_site_id] - 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] #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 (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 #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 - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [web_site_id] #9 + CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] + CometProject [sales_price,profit,return_amt,net_loss,web_site_id] + CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] + CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + 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,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + 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,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] + CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #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_web_site_sk,ws_order_number,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange [web_site_sk,web_site_id] #11 + CometFilter [web_site_sk,web_site_id] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + WholeStageCodegen (11) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel] #12 + RowToColumnar + WholeStageCodegen (10) + 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] + ColumnarToRow + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + WholeStageCodegen (17) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange #13 + RowToColumnar + WholeStageCodegen (16) + 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] + ColumnarToRow + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 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 fc156f98e..b662f22fb 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,44 +1,42 @@ == Physical Plan == -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) +* ColumnarToRow (38) ++- CometTakeOrderedAndProject (37) + +- CometFilter (36) + +- CometHashAggregate (35) + +- CometColumnarExchange (34) + +- CometHashAggregate (33) + +- CometProject (32) + +- CometBroadcastHashJoin (31) + :- 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) + +- CometBroadcastExchange (30) + +- CometProject (29) + +- CometBroadcastHashJoin (28) + :- CometFilter (20) + : +- CometScan parquet spark_catalog.default.item (19) + +- CometBroadcastExchange (27) + +- CometFilter (26) + +- CometHashAggregate (25) + +- CometColumnarExchange (24) + +- CometHashAggregate (23) + +- CometFilter (22) + +- CometScan parquet spark_catalog.default.item (21) (1) Scan parquet spark_catalog.default.customer_address @@ -129,186 +127,170 @@ Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight 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] - -(20) Scan parquet spark_catalog.default.item +(19) 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 -(21) CometFilter +(20) 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)) -(22) ColumnarToRow [codegen id : 3] -Input [3]: [i_item_sk#13, i_current_price#14, i_category#15] - -(23) Scan parquet spark_catalog.default.item +(21) 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 -(24) CometFilter +(22) CometFilter Input [2]: [i_current_price#16, i_category#17] Condition : isnotnull(i_category#17) -(25) CometHashAggregate +(23) CometHashAggregate Input [2]: [i_current_price#16, i_category#17] Keys [1]: [i_category#17] Functions [1]: [partial_avg(UnscaledValue(i_current_price#16))] -(26) ColumnarToRow [codegen id : 1] +(24) CometColumnarExchange Input [3]: [i_category#17, sum#18, count#19] +Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(27) Exchange -Input [3]: [i_category#17, sum#18, count#19] -Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, [plan_id=1] - -(28) HashAggregate [codegen id : 2] +(25) CometHashAggregate 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] -(29) Filter [codegen id : 2] -Input [2]: [avg(i_current_price)#21, i_category#17] -Condition : isnotnull(avg(i_current_price)#21) +(26) CometFilter +Input [2]: [avg(i_current_price)#20, i_category#17] +Condition : isnotnull(avg(i_current_price)#20) -(30) BroadcastExchange -Input [2]: [avg(i_current_price)#21, i_category#17] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=2] +(27) CometBroadcastExchange +Input [2]: [avg(i_current_price)#20, i_category#17] +Arguments: [avg(i_current_price)#20, i_category#17] -(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#14 as decimal(14,7)) > (1.2 * avg(i_current_price)#21)) +(28) CometBroadcastHashJoin +Left output [3]: [i_item_sk#13, i_current_price#14, i_category#15] +Right output [2]: [avg(i_current_price)#20, i_category#17] +Arguments: [i_category#15], [i_category#17], Inner, (cast(i_current_price#14 as decimal(14,7)) > (1.2 * avg(i_current_price)#20)), BuildRight -(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] +(29) CometProject +Input [5]: [i_item_sk#13, i_current_price#14, i_category#15, avg(i_current_price)#20, i_category#17] +Arguments: [i_item_sk#13], [i_item_sk#13] -(33) BroadcastExchange +(30) CometBroadcastExchange Input [1]: [i_item_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: [i_item_sk#13] -(34) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#5] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None +(31) CometBroadcastHashJoin +Left output [2]: [ca_state#2, ss_item_sk#5] +Right output [1]: [i_item_sk#13] +Arguments: [ss_item_sk#5], [i_item_sk#13], Inner, BuildRight -(35) Project [codegen id : 4] -Output [1]: [ca_state#2] +(32) CometProject Input [3]: [ca_state#2, ss_item_sk#5, i_item_sk#13] +Arguments: [ca_state#2], [ca_state#2] -(36) HashAggregate [codegen id : 4] +(33) CometHashAggregate Input [1]: [ca_state#2] Keys [1]: [ca_state#2] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#22] -Results [2]: [ca_state#2, count#23] -(37) Exchange -Input [2]: [ca_state#2, count#23] -Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(34) CometColumnarExchange +Input [2]: [ca_state#2, count#21] +Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(38) HashAggregate [codegen id : 5] -Input [2]: [ca_state#2, count#23] +(35) CometHashAggregate +Input [2]: [ca_state#2, count#21] Keys [1]: [ca_state#2] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#24] -Results [3]: [ca_state#2 AS state#25, count(1)#24 AS cnt#26, ca_state#2] -(39) Filter [codegen id : 5] -Input [3]: [state#25, cnt#26, ca_state#2] -Condition : (cnt#26 >= 10) +(36) CometFilter +Input [3]: [state#22, cnt#23, ca_state#2] +Condition : (cnt#23 >= 10) -(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] +(37) CometTakeOrderedAndProject +Input [3]: [state#22, cnt#23, ca_state#2] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[cnt#23 ASC NULLS FIRST,ca_state#2 ASC NULLS FIRST], output=[state#22,cnt#23]), [state#22, cnt#23], 100, [cnt#23 ASC NULLS FIRST, ca_state#2 ASC NULLS FIRST], [state#22, cnt#23] + +(38) ColumnarToRow [codegen id : 1] +Input [2]: [state#22, cnt#23] ===== Subqueries ===== 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) +BroadcastExchange (43) ++- * ColumnarToRow (42) + +- CometProject (41) + +- CometFilter (40) + +- CometScan parquet spark_catalog.default.date_dim (39) -(41) Scan parquet spark_catalog.default.date_dim +(39) 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 -(42) CometFilter +(40) 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)) -(43) CometProject +(41) CometProject Input [2]: [d_date_sk#9, d_month_seq#10] Arguments: [d_date_sk#9], [d_date_sk#9] -(44) ColumnarToRow [codegen id : 1] +(42) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(45) BroadcastExchange +(43) BroadcastExchange Input [1]: [d_date_sk#9] -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 = 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) +Subquery:2 Hosting operator id = 40 Hosting Expression = Subquery scalar-subquery#11, [id=#12] +* ColumnarToRow (50) ++- CometHashAggregate (49) + +- CometColumnarExchange (48) + +- CometHashAggregate (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan parquet spark_catalog.default.date_dim (44) -(46) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#27, d_year#28, d_moy#29] +(44) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_month_seq#24, d_year#25, d_moy#26] 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 -(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)) +(45) CometFilter +Input [3]: [d_month_seq#24, d_year#25, d_moy#26] +Condition : (((isnotnull(d_year#25) AND isnotnull(d_moy#26)) AND (d_year#25 = 2000)) AND (d_moy#26 = 1)) -(48) CometProject -Input [3]: [d_month_seq#27, d_year#28, d_moy#29] -Arguments: [d_month_seq#27], [d_month_seq#27] +(46) CometProject +Input [3]: [d_month_seq#24, d_year#25, d_moy#26] +Arguments: [d_month_seq#24], [d_month_seq#24] -(49) CometHashAggregate -Input [1]: [d_month_seq#27] -Keys [1]: [d_month_seq#27] +(47) CometHashAggregate +Input [1]: [d_month_seq#24] +Keys [1]: [d_month_seq#24] Functions: [] -(50) ColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#27] - -(51) Exchange -Input [1]: [d_month_seq#27] -Arguments: hashpartitioning(d_month_seq#27, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(48) CometColumnarExchange +Input [1]: [d_month_seq#24] +Arguments: hashpartitioning(d_month_seq#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(52) HashAggregate [codegen id : 2] -Input [1]: [d_month_seq#27] -Keys [1]: [d_month_seq#27] +(49) CometHashAggregate +Input [1]: [d_month_seq#24] +Keys [1]: [d_month_seq#24] Functions: [] -Aggregate Attributes: [] -Results [1]: [d_month_seq#27] + +(50) ColumnarToRow [codegen id : 1] +Input [1]: [d_month_seq#24] 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 cfb74b3b2..b95744087 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,73 +1,59 @@ -TakeOrderedAndProject [cnt,ca_state,state] - WholeStageCodegen (5) - Filter [cnt] - HashAggregate [ca_state,count] [count(1),state,cnt,count] - InputAdapter - Exchange [ca_state] #1 - WholeStageCodegen (4) - HashAggregate [ca_state] [count,count] - Project [ca_state] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometProject [ca_state,ss_item_sk] - CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] - CometProject [ca_state,ss_item_sk,ss_sold_date_sk] - CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] - CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_customer_sk,ss_sold_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 #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [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_month_seq,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 [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [d_month_seq] #2 - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - Project [i_item_sk] - BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_current_price,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - InputAdapter - 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] #9 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_category,sum,count,i_current_price] - CometFilter [i_current_price,i_category] - CometScan parquet spark_catalog.default.item [i_current_price,i_category] +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [state,cnt,ca_state] + CometFilter [state,cnt,ca_state] + CometHashAggregate [state,cnt,ca_state,count,count(1)] + CometColumnarExchange [ca_state] #1 + CometHashAggregate [ca_state,count] + CometProject [ca_state] + CometBroadcastHashJoin [ca_state,ss_item_sk,i_item_sk] + CometProject [ca_state,ss_item_sk] + CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] + CometProject [ca_state,ss_item_sk,ss_sold_date_sk] + CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] + CometProject [ca_state,c_customer_sk] + CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_customer_sk,ss_sold_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 #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + Subquery #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [d_month_seq] + CometColumnarExchange [d_month_seq] #5 + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_month_seq,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 [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + ReusedSubquery [d_month_seq] #2 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk] #7 + CometProject [i_item_sk] + CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] + CometFilter [i_item_sk,i_current_price,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] + CometBroadcastExchange [avg(i_current_price),i_category] #8 + CometFilter [avg(i_current_price),i_category] + CometHashAggregate [avg(i_current_price),i_category,sum,count,avg(UnscaledValue(i_current_price))] + CometColumnarExchange [i_category] #9 + CometHashAggregate [i_category,sum,count,i_current_price] + CometFilter [i_current_price,i_category] + CometScan parquet spark_catalog.default.item [i_current_price,i_category] 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 4088fb409..174590641 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 @@ -1,185 +1,180 @@ == Physical Plan == -* Sort (181) -+- Exchange (180) - +- * Project (179) - +- * SortMergeJoin Inner (178) - :- * Sort (110) - : +- Exchange (109) - : +- * HashAggregate (108) - : +- * HashAggregate (107) - : +- * Project (106) - : +- * BroadcastHashJoin Inner BuildRight (105) - : :- * Project (99) - : : +- * BroadcastHashJoin Inner BuildRight (98) - : : :- * Project (96) - : : : +- * BroadcastHashJoin Inner BuildRight (95) - : : : :- * Project (90) - : : : : +- * BroadcastHashJoin Inner BuildRight (89) - : : : : :- * Project (87) - : : : : : +- * BroadcastHashJoin Inner BuildRight (86) - : : : : : :- * Project (81) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (80) - : : : : : : :- * Project (78) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (77) - : : : : : : : :- * Project (72) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (71) - : : : : : : : : :- * Project (66) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (65) - : : : : : : : : : :- * Project (63) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (62) - : : : : : : : : : : :- * Project (57) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (56) - : : : : : : : : : : : :- * Project (54) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (53) - : : : : : : : : : : : : :- * Project (48) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (47) - : : : : : : : : : : : : : :- * Project (42) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : : : : : : : : : : : :- * Project (36) - : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (35) - : : : : : : : : : : : : : : : :- * Project (33) - : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (32) - : : : : : : : : : : : : : : : : :- * Sort (11) - : : : : : : : : : : : : : : : : : +- Exchange (10) - : : : : : : : : : : : : : : : : : +- * ColumnarToRow (9) - : : : : : : : : : : : : : : : : : +- CometProject (8) - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) - : : : : : : : : : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- CometProject (6) - : : : : : : : : : : : : : : : : : +- CometFilter (5) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : : : : : : : : : : : : : : : +- * Sort (31) - : : : : : : : : : : : : : : : : +- * Project (30) - : : : : : : : : : : : : : : : : +- * Filter (29) - : : : : : : : : : : : : : : : : +- * HashAggregate (28) - : : : : : : : : : : : : : : : : +- Exchange (27) - : : : : : : : : : : : : : : : : +- * HashAggregate (26) - : : : : : : : : : : : : : : : : +- * Project (25) - : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (24) - : : : : : : : : : : : : : : : : :- * Sort (17) - : : : : : : : : : : : : : : : : : +- Exchange (16) - : : : : : : : : : : : : : : : : : +- * ColumnarToRow (15) - : : : : : : : : : : : : : : : : : +- CometProject (14) - : : : : : : : : : : : : : : : : : +- CometFilter (13) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (12) - : : : : : : : : : : : : : : : : +- * Sort (23) - : : : : : : : : : : : : : : : : +- Exchange (22) - : : : : : : : : : : : : : : : : +- * ColumnarToRow (21) - : : : : : : : : : : : : : : : : +- CometProject (20) - : : : : : : : : : : : : : : : : +- CometFilter (19) - : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (18) - : : : : : : : : : : : : : : : +- ReusedExchange (34) - : : : : : : : : : : : : : : +- BroadcastExchange (40) - : : : : : : : : : : : : : : +- * ColumnarToRow (39) - : : : : : : : : : : : : : : +- CometFilter (38) - : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (37) - : : : : : : : : : : : : : +- BroadcastExchange (46) - : : : : : : : : : : : : : +- * ColumnarToRow (45) - : : : : : : : : : : : : : +- CometFilter (44) - : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (43) - : : : : : : : : : : : : +- BroadcastExchange (52) - : : : : : : : : : : : : +- * ColumnarToRow (51) - : : : : : : : : : : : : +- CometFilter (50) - : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (49) - : : : : : : : : : : : +- ReusedExchange (55) - : : : : : : : : : : +- BroadcastExchange (61) - : : : : : : : : : : +- * ColumnarToRow (60) - : : : : : : : : : : +- CometFilter (59) - : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (58) - : : : : : : : : : +- ReusedExchange (64) - : : : : : : : : +- BroadcastExchange (70) - : : : : : : : : +- * ColumnarToRow (69) - : : : : : : : : +- CometFilter (68) - : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (67) - : : : : : : : +- BroadcastExchange (76) - : : : : : : : +- * ColumnarToRow (75) - : : : : : : : +- CometFilter (74) - : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (73) - : : : : : : +- ReusedExchange (79) - : : : : : +- BroadcastExchange (85) - : : : : : +- * ColumnarToRow (84) - : : : : : +- CometFilter (83) - : : : : : +- CometScan parquet spark_catalog.default.customer_address (82) - : : : : +- ReusedExchange (88) - : : : +- BroadcastExchange (94) - : : : +- * ColumnarToRow (93) - : : : +- CometFilter (92) - : : : +- CometScan parquet spark_catalog.default.income_band (91) - : : +- ReusedExchange (97) - : +- BroadcastExchange (104) - : +- * ColumnarToRow (103) - : +- CometProject (102) - : +- CometFilter (101) - : +- CometScan parquet spark_catalog.default.item (100) - +- * Sort (177) - +- Exchange (176) - +- * HashAggregate (175) - +- * HashAggregate (174) - +- * Project (173) - +- * BroadcastHashJoin Inner BuildRight (172) - :- * Project (170) - : +- * BroadcastHashJoin Inner BuildRight (169) - : :- * Project (167) - : : +- * BroadcastHashJoin Inner BuildRight (166) - : : :- * Project (164) - : : : +- * BroadcastHashJoin Inner BuildRight (163) - : : : :- * Project (161) - : : : : +- * BroadcastHashJoin Inner BuildRight (160) - : : : : :- * Project (158) - : : : : : +- * BroadcastHashJoin Inner BuildRight (157) - : : : : : :- * Project (155) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (154) - : : : : : : :- * Project (152) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (151) - : : : : : : : :- * Project (149) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (148) - : : : : : : : : :- * Project (146) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (145) - : : : : : : : : : :- * Project (143) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (142) - : : : : : : : : : : :- * Project (140) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (139) - : : : : : : : : : : : :- * Project (137) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (136) - : : : : : : : : : : : : :- * Project (134) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (133) - : : : : : : : : : : : : : :- * Project (131) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (130) - : : : : : : : : : : : : : : :- * Project (128) - : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (127) - : : : : : : : : : : : : : : : :- * Sort (121) - : : : : : : : : : : : : : : : : +- Exchange (120) - : : : : : : : : : : : : : : : : +- * ColumnarToRow (119) - : : : : : : : : : : : : : : : : +- CometProject (118) - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (117) - : : : : : : : : : : : : : : : : :- CometBroadcastExchange (113) - : : : : : : : : : : : : : : : : : +- CometFilter (112) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (111) - : : : : : : : : : : : : : : : : +- CometProject (116) - : : : : : : : : : : : : : : : : +- CometFilter (115) - : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (114) - : : : : : : : : : : : : : : : +- * Sort (126) - : : : : : : : : : : : : : : : +- * Project (125) - : : : : : : : : : : : : : : : +- * Filter (124) - : : : : : : : : : : : : : : : +- * HashAggregate (123) - : : : : : : : : : : : : : : : +- ReusedExchange (122) - : : : : : : : : : : : : : : +- ReusedExchange (129) - : : : : : : : : : : : : : +- ReusedExchange (132) - : : : : : : : : : : : : +- ReusedExchange (135) - : : : : : : : : : : : +- ReusedExchange (138) - : : : : : : : : : : +- ReusedExchange (141) - : : : : : : : : : +- ReusedExchange (144) - : : : : : : : : +- ReusedExchange (147) - : : : : : : : +- ReusedExchange (150) - : : : : : : +- ReusedExchange (153) - : : : : : +- ReusedExchange (156) - : : : : +- ReusedExchange (159) - : : : +- ReusedExchange (162) - : : +- ReusedExchange (165) - : +- ReusedExchange (168) - +- ReusedExchange (171) +* ColumnarToRow (176) ++- CometSort (175) + +- CometColumnarExchange (174) + +- RowToColumnar (173) + +- * Project (172) + +- * SortMergeJoin Inner (171) + :- * ColumnarToRow (101) + : +- CometSort (100) + : +- CometColumnarExchange (99) + : +- CometHashAggregate (98) + : +- CometHashAggregate (97) + : +- CometProject (96) + : +- CometBroadcastHashJoin (95) + : :- CometProject (90) + : : +- CometBroadcastHashJoin (89) + : : :- CometProject (87) + : : : +- CometBroadcastHashJoin (86) + : : : :- CometProject (82) + : : : : +- CometBroadcastHashJoin (81) + : : : : :- CometProject (79) + : : : : : +- CometBroadcastHashJoin (78) + : : : : : :- CometProject (74) + : : : : : : +- CometBroadcastHashJoin (73) + : : : : : : :- CometProject (71) + : : : : : : : +- CometBroadcastHashJoin (70) + : : : : : : : :- CometProject (66) + : : : : : : : : +- CometBroadcastHashJoin (65) + : : : : : : : : :- CometProject (61) + : : : : : : : : : +- CometBroadcastHashJoin (60) + : : : : : : : : : :- CometProject (58) + : : : : : : : : : : +- CometBroadcastHashJoin (57) + : : : : : : : : : : :- CometProject (53) + : : : : : : : : : : : +- CometBroadcastHashJoin (52) + : : : : : : : : : : : :- CometProject (50) + : : : : : : : : : : : : +- CometBroadcastHashJoin (49) + : : : : : : : : : : : : :- CometProject (45) + : : : : : : : : : : : : : +- CometBroadcastHashJoin (44) + : : : : : : : : : : : : : :- CometProject (40) + : : : : : : : : : : : : : : +- CometBroadcastHashJoin (39) + : : : : : : : : : : : : : : :- CometProject (35) + : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (34) + : : : : : : : : : : : : : : : :- CometProject (30) + : : : : : : : : : : : : : : : : +- CometSortMergeJoin (29) + : : : : : : : : : : : : : : : : :- CometSort (10) + : : : : : : : : : : : : : : : : : +- CometColumnarExchange (9) + : : : : : : : : : : : : : : : : : +- CometProject (8) + : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) + : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) + : : : : : : : : : : : : : : : : : : +- CometFilter (2) + : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : : : : : : : : : : : : +- CometProject (6) + : : : : : : : : : : : : : : : : : +- CometFilter (5) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : : : : : : : : : : : : : : : +- CometSort (28) + : : : : : : : : : : : : : : : : +- CometProject (27) + : : : : : : : : : : : : : : : : +- CometFilter (26) + : : : : : : : : : : : : : : : : +- CometHashAggregate (25) + : : : : : : : : : : : : : : : : +- CometColumnarExchange (24) + : : : : : : : : : : : : : : : : +- CometHashAggregate (23) + : : : : : : : : : : : : : : : : +- CometProject (22) + : : : : : : : : : : : : : : : : +- CometSortMergeJoin (21) + : : : : : : : : : : : : : : : : :- CometSort (15) + : : : : : : : : : : : : : : : : : +- CometColumnarExchange (14) + : : : : : : : : : : : : : : : : : +- CometProject (13) + : : : : : : : : : : : : : : : : : +- CometFilter (12) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (11) + : : : : : : : : : : : : : : : : +- CometSort (20) + : : : : : : : : : : : : : : : : +- CometColumnarExchange (19) + : : : : : : : : : : : : : : : : +- CometProject (18) + : : : : : : : : : : : : : : : : +- CometFilter (17) + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (16) + : : : : : : : : : : : : : : : +- CometBroadcastExchange (33) + : : : : : : : : : : : : : : : +- CometFilter (32) + : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (31) + : : : : : : : : : : : : : : +- CometBroadcastExchange (38) + : : : : : : : : : : : : : : +- CometFilter (37) + : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (36) + : : : : : : : : : : : : : +- CometBroadcastExchange (43) + : : : : : : : : : : : : : +- CometFilter (42) + : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (41) + : : : : : : : : : : : : +- CometBroadcastExchange (48) + : : : : : : : : : : : : +- CometFilter (47) + : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (46) + : : : : : : : : : : : +- ReusedExchange (51) + : : : : : : : : : : +- CometBroadcastExchange (56) + : : : : : : : : : : +- CometFilter (55) + : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (54) + : : : : : : : : : +- ReusedExchange (59) + : : : : : : : : +- CometBroadcastExchange (64) + : : : : : : : : +- CometFilter (63) + : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (62) + : : : : : : : +- CometBroadcastExchange (69) + : : : : : : : +- CometFilter (68) + : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (67) + : : : : : : +- ReusedExchange (72) + : : : : : +- CometBroadcastExchange (77) + : : : : : +- CometFilter (76) + : : : : : +- CometScan parquet spark_catalog.default.customer_address (75) + : : : : +- ReusedExchange (80) + : : : +- CometBroadcastExchange (85) + : : : +- CometFilter (84) + : : : +- CometScan parquet spark_catalog.default.income_band (83) + : : +- ReusedExchange (88) + : +- CometBroadcastExchange (94) + : +- CometProject (93) + : +- CometFilter (92) + : +- CometScan parquet spark_catalog.default.item (91) + +- * ColumnarToRow (170) + +- CometSort (169) + +- CometColumnarExchange (168) + +- CometHashAggregate (167) + +- CometHashAggregate (166) + +- CometProject (165) + +- CometBroadcastHashJoin (164) + :- CometProject (162) + : +- CometBroadcastHashJoin (161) + : :- CometProject (159) + : : +- CometBroadcastHashJoin (158) + : : :- CometProject (156) + : : : +- CometBroadcastHashJoin (155) + : : : :- CometProject (153) + : : : : +- CometBroadcastHashJoin (152) + : : : : :- CometProject (150) + : : : : : +- CometBroadcastHashJoin (149) + : : : : : :- CometProject (147) + : : : : : : +- CometBroadcastHashJoin (146) + : : : : : : :- CometProject (144) + : : : : : : : +- CometBroadcastHashJoin (143) + : : : : : : : :- CometProject (141) + : : : : : : : : +- CometBroadcastHashJoin (140) + : : : : : : : : :- CometProject (138) + : : : : : : : : : +- CometBroadcastHashJoin (137) + : : : : : : : : : :- CometProject (135) + : : : : : : : : : : +- CometBroadcastHashJoin (134) + : : : : : : : : : : :- CometProject (132) + : : : : : : : : : : : +- CometBroadcastHashJoin (131) + : : : : : : : : : : : :- CometProject (129) + : : : : : : : : : : : : +- CometBroadcastHashJoin (128) + : : : : : : : : : : : : :- CometProject (126) + : : : : : : : : : : : : : +- CometBroadcastHashJoin (125) + : : : : : : : : : : : : : :- CometProject (123) + : : : : : : : : : : : : : : +- CometBroadcastHashJoin (122) + : : : : : : : : : : : : : : :- CometProject (118) + : : : : : : : : : : : : : : : +- CometSortMergeJoin (117) + : : : : : : : : : : : : : : : :- CometSort (111) + : : : : : : : : : : : : : : : : +- CometColumnarExchange (110) + : : : : : : : : : : : : : : : : +- CometProject (109) + : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (108) + : : : : : : : : : : : : : : : : :- CometBroadcastExchange (104) + : : : : : : : : : : : : : : : : : +- CometFilter (103) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (102) + : : : : : : : : : : : : : : : : +- CometProject (107) + : : : : : : : : : : : : : : : : +- CometFilter (106) + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (105) + : : : : : : : : : : : : : : : +- CometSort (116) + : : : : : : : : : : : : : : : +- CometProject (115) + : : : : : : : : : : : : : : : +- CometFilter (114) + : : : : : : : : : : : : : : : +- CometHashAggregate (113) + : : : : : : : : : : : : : : : +- ReusedExchange (112) + : : : : : : : : : : : : : : +- CometBroadcastExchange (121) + : : : : : : : : : : : : : : +- CometFilter (120) + : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (119) + : : : : : : : : : : : : : +- ReusedExchange (124) + : : : : : : : : : : : : +- ReusedExchange (127) + : : : : : : : : : : : +- ReusedExchange (130) + : : : : : : : : : : +- ReusedExchange (133) + : : : : : : : : : +- ReusedExchange (136) + : : : : : : : : +- ReusedExchange (139) + : : : : : : : +- ReusedExchange (142) + : : : : : : +- ReusedExchange (145) + : : : : : +- ReusedExchange (148) + : : : : +- ReusedExchange (151) + : : : +- ReusedExchange (154) + : : +- ReusedExchange (157) + : +- ReusedExchange (160) + +- ReusedExchange (163) (1) Scan parquet spark_catalog.default.store_sales @@ -222,843 +217,793 @@ Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number# 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] Arguments: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -(9) ColumnarToRow [codegen id : 1] +(9) CometColumnarExchange Input [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(10) Exchange +(10) CometSort Input [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1 ASC NULLS FIRST] -(11) Sort [codegen id : 2] -Input [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 - -(12) Scan parquet spark_catalog.default.catalog_sales +(11) Scan parquet spark_catalog.default.catalog_sales Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_order_number)] ReadSchema: struct -(13) CometFilter +(12) CometFilter Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) -(14) CometProject +(13) CometProject Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -(15) ColumnarToRow [codegen id : 3] -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] - -(16) Exchange +(14) CometColumnarExchange Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(17) Sort [codegen id : 4] +(15) CometSort Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST], false, 0 +Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST] -(18) Scan parquet spark_catalog.default.catalog_returns +(16) Scan parquet spark_catalog.default.catalog_returns Output [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(19) CometFilter +(17) CometFilter Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) -(20) CometProject +(18) CometProject Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -(21) ColumnarToRow [codegen id : 5] +(19) CometColumnarExchange Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) Exchange +(20) CometSort Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST] -(23) Sort [codegen id : 6] -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST], false, 0 +(21) CometSortMergeJoin +Left output [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] +Right output [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number#22], Inner -(24) SortMergeJoin [codegen id : 7] -Left keys [2]: [cs_item_sk#17, cs_order_number#18] -Right keys [2]: [cr_item_sk#21, cr_order_number#22] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 7] -Output [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +(22) CometProject Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +Arguments: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -(26) HashAggregate [codegen id : 7] +(23) CometHashAggregate Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Keys [1]: [cs_item_sk#17] Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] -Aggregate Attributes [3]: [sum#27, sum#28, isEmpty#29] -Results [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] -(27) Exchange -Input [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(24) CometColumnarExchange +Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] +Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(28) HashAggregate [codegen id : 8] -Input [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] +(25) CometHashAggregate +Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] Keys [1]: [cs_item_sk#17] Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#19))#33, sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))#34] -Results [3]: [cs_item_sk#17, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#19))#33,17,2) AS sale#35, sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))#34 AS refund#36] -(29) Filter [codegen id : 8] -Input [3]: [cs_item_sk#17, sale#35, refund#36] -Condition : ((isnotnull(sale#35) AND isnotnull(refund#36)) AND (cast(sale#35 as decimal(21,2)) > (2 * refund#36))) +(26) CometFilter +Input [3]: [cs_item_sk#17, sale#30, refund#31] +Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) -(30) Project [codegen id : 8] -Output [1]: [cs_item_sk#17] -Input [3]: [cs_item_sk#17, sale#35, refund#36] +(27) CometProject +Input [3]: [cs_item_sk#17, sale#30, refund#31] +Arguments: [cs_item_sk#17], [cs_item_sk#17] -(31) Sort [codegen id : 8] +(28) CometSort Input [1]: [cs_item_sk#17] -Arguments: [cs_item_sk#17 ASC NULLS FIRST], false, 0 +Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] -(32) SortMergeJoin [codegen id : 24] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [cs_item_sk#17] -Join type: Inner -Join condition: None +(29) CometSortMergeJoin +Left output [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Right output [1]: [cs_item_sk#17] +Arguments: [ss_item_sk#1], [cs_item_sk#17], Inner -(33) Project [codegen id : 24] -Output [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +(30) CometProject Input [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#17] +Arguments: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] + +(31) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#32, d_year#33] +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 -(34) ReusedExchange [Reuses operator id: 185] -Output [2]: [d_date_sk#37, d_year#38] +(32) CometFilter +Input [2]: [d_date_sk#32, d_year#33] +Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(35) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#37] -Join type: Inner -Join condition: None +(33) CometBroadcastExchange +Input [2]: [d_date_sk#32, d_year#33] +Arguments: [d_date_sk#32, d_year#33] + +(34) CometBroadcastHashJoin +Left output [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Right output [2]: [d_date_sk#32, d_year#33] +Arguments: [ss_sold_date_sk#12], [d_date_sk#32], Inner, BuildRight -(36) Project [codegen id : 24] -Output [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38] -Input [13]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#37, d_year#38] +(35) CometProject +Input [13]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#32, d_year#33] +Arguments: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33], [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] -(37) Scan parquet spark_catalog.default.store -Output [3]: [s_store_sk#39, s_store_name#40, s_zip#41] +(36) Scan parquet spark_catalog.default.store +Output [3]: [s_store_sk#34, s_store_name#35, s_zip#36] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_zip)] ReadSchema: struct -(38) CometFilter -Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] -Condition : ((isnotnull(s_store_sk#39) AND isnotnull(s_store_name#40)) AND isnotnull(s_zip#41)) +(37) CometFilter +Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] +Condition : ((isnotnull(s_store_sk#34) AND isnotnull(s_store_name#35)) AND isnotnull(s_zip#36)) -(39) ColumnarToRow [codegen id : 10] -Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] +(38) CometBroadcastExchange +Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] +Arguments: [s_store_sk#34, s_store_name#35, s_zip#36] -(40) BroadcastExchange -Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +(39) CometBroadcastHashJoin +Left output [11]: [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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] +Right output [3]: [s_store_sk#34, s_store_name#35, s_zip#36] +Arguments: [ss_store_sk#6], [s_store_sk#34], Inner, BuildRight -(41) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_store_sk#6] -Right keys [1]: [s_store_sk#39] -Join type: Inner -Join condition: None +(40) 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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_sk#34, s_store_name#35, s_zip#36] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36] -(42) Project [codegen id : 24] -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41] -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_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_sk#39, s_store_name#40, s_zip#41] - -(43) Scan parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] +(41) Scan parquet spark_catalog.default.customer +Output [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(44) CometFilter -Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] -Condition : (((((isnotnull(c_customer_sk#42) AND isnotnull(c_first_sales_date_sk#47)) AND isnotnull(c_first_shipto_date_sk#46)) AND isnotnull(c_current_cdemo_sk#43)) AND isnotnull(c_current_hdemo_sk#44)) AND isnotnull(c_current_addr_sk#45)) - -(45) ColumnarToRow [codegen id : 11] -Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] +(42) CometFilter +Input [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +Condition : (((((isnotnull(c_customer_sk#37) AND isnotnull(c_first_sales_date_sk#42)) AND isnotnull(c_first_shipto_date_sk#41)) AND isnotnull(c_current_cdemo_sk#38)) AND isnotnull(c_current_hdemo_sk#39)) AND isnotnull(c_current_addr_sk#40)) -(46) BroadcastExchange -Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +(43) CometBroadcastExchange +Input [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +Arguments: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] -(47) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#42] -Join type: Inner -Join condition: None +(44) 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_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36] +Right output [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +Arguments: [ss_customer_sk#2], [c_customer_sk#37], Inner, BuildRight -(48) Project [codegen id : 24] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] +(45) CometProject +Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] -(49) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#48, d_year#49] +(46) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#43, d_year#44] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct -(50) CometFilter -Input [2]: [d_date_sk#48, d_year#49] -Condition : isnotnull(d_date_sk#48) +(47) CometFilter +Input [2]: [d_date_sk#43, d_year#44] +Condition : isnotnull(d_date_sk#43) -(51) ColumnarToRow [codegen id : 12] -Input [2]: [d_date_sk#48, d_year#49] +(48) CometBroadcastExchange +Input [2]: [d_date_sk#43, d_year#44] +Arguments: [d_date_sk#43, d_year#44] -(52) BroadcastExchange -Input [2]: [d_date_sk#48, d_year#49] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] - -(53) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [c_first_sales_date_sk#47] -Right keys [1]: [d_date_sk#48] -Join type: Inner -Join condition: None +(49) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +Right output [2]: [d_date_sk#43, d_year#44] +Arguments: [c_first_sales_date_sk#42], [d_date_sk#43], Inner, BuildRight -(54) Project [codegen id : 24] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, d_year#49] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47, d_date_sk#48, d_year#49] +(50) CometProject +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42, d_date_sk#43, d_year#44] +Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, d_year#44], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, d_year#44] -(55) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#50, d_year#51] +(51) ReusedExchange [Reuses operator id: 48] +Output [2]: [d_date_sk#45, d_year#46] -(56) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [c_first_shipto_date_sk#46] -Right keys [1]: [d_date_sk#50] -Join type: Inner -Join condition: None +(52) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, d_year#44] +Right output [2]: [d_date_sk#45, d_year#46] +Arguments: [c_first_shipto_date_sk#41], [d_date_sk#45], Inner, BuildRight -(57) Project [codegen id : 24] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, d_year#49, d_date_sk#50, d_year#51] +(53) CometProject +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, d_year#44, d_date_sk#45, d_year#46] +Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] -(58) Scan parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#52, cd_marital_status#53] +(54) Scan parquet spark_catalog.default.customer_demographics +Output [2]: [cd_demo_sk#47, cd_marital_status#48] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status)] ReadSchema: struct -(59) CometFilter -Input [2]: [cd_demo_sk#52, cd_marital_status#53] -Condition : (isnotnull(cd_demo_sk#52) AND isnotnull(cd_marital_status#53)) - -(60) ColumnarToRow [codegen id : 14] -Input [2]: [cd_demo_sk#52, cd_marital_status#53] +(55) CometFilter +Input [2]: [cd_demo_sk#47, cd_marital_status#48] +Condition : (isnotnull(cd_demo_sk#47) AND isnotnull(cd_marital_status#48)) -(61) BroadcastExchange -Input [2]: [cd_demo_sk#52, cd_marital_status#53] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +(56) CometBroadcastExchange +Input [2]: [cd_demo_sk#47, cd_marital_status#48] +Arguments: [cd_demo_sk#47, cd_marital_status#48] -(62) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#52] -Join type: Inner -Join condition: None +(57) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] +Right output [2]: [cd_demo_sk#47, cd_marital_status#48] +Arguments: [ss_cdemo_sk#3], [cd_demo_sk#47], Inner, BuildRight -(63) Project [codegen id : 24] -Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_marital_status#53] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_demo_sk#52, cd_marital_status#53] +(58) CometProject +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_demo_sk#47, cd_marital_status#48] +Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_marital_status#48], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_marital_status#48] -(64) ReusedExchange [Reuses operator id: 61] -Output [2]: [cd_demo_sk#54, cd_marital_status#55] +(59) ReusedExchange [Reuses operator id: 56] +Output [2]: [cd_demo_sk#49, cd_marital_status#50] -(65) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [c_current_cdemo_sk#43] -Right keys [1]: [cd_demo_sk#54] -Join type: Inner -Join condition: NOT (cd_marital_status#53 = cd_marital_status#55) +(60) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_marital_status#48] +Right output [2]: [cd_demo_sk#49, cd_marital_status#50] +Arguments: [c_current_cdemo_sk#38], [cd_demo_sk#49], Inner, NOT (cd_marital_status#48 = cd_marital_status#50), BuildRight -(66) Project [codegen id : 24] -Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_marital_status#53, cd_demo_sk#54, cd_marital_status#55] +(61) CometProject +Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_marital_status#48, cd_demo_sk#49, cd_marital_status#50] +Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] -(67) Scan parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#56] +(62) Scan parquet spark_catalog.default.promotion +Output [1]: [p_promo_sk#51] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_promo_sk)] ReadSchema: struct -(68) CometFilter -Input [1]: [p_promo_sk#56] -Condition : isnotnull(p_promo_sk#56) +(63) CometFilter +Input [1]: [p_promo_sk#51] +Condition : isnotnull(p_promo_sk#51) -(69) ColumnarToRow [codegen id : 16] -Input [1]: [p_promo_sk#56] +(64) CometBroadcastExchange +Input [1]: [p_promo_sk#51] +Arguments: [p_promo_sk#51] -(70) BroadcastExchange -Input [1]: [p_promo_sk#56] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] +(65) CometBroadcastHashJoin +Left output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] +Right output [1]: [p_promo_sk#51] +Arguments: [ss_promo_sk#7], [p_promo_sk#51], Inner, BuildRight -(71) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_promo_sk#7] -Right keys [1]: [p_promo_sk#56] -Join type: Inner -Join condition: None - -(72) Project [codegen id : 24] -Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, p_promo_sk#56] +(66) CometProject +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, p_promo_sk#51] +Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] -(73) Scan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#57, hd_income_band_sk#58] +(67) Scan parquet spark_catalog.default.household_demographics +Output [2]: [hd_demo_sk#52, hd_income_band_sk#53] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] ReadSchema: struct -(74) CometFilter -Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] -Condition : (isnotnull(hd_demo_sk#57) AND isnotnull(hd_income_band_sk#58)) - -(75) ColumnarToRow [codegen id : 17] -Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] +(68) CometFilter +Input [2]: [hd_demo_sk#52, hd_income_band_sk#53] +Condition : (isnotnull(hd_demo_sk#52) AND isnotnull(hd_income_band_sk#53)) -(76) BroadcastExchange -Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] +(69) CometBroadcastExchange +Input [2]: [hd_demo_sk#52, hd_income_band_sk#53] +Arguments: [hd_demo_sk#52, hd_income_band_sk#53] -(77) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_hdemo_sk#4] -Right keys [1]: [hd_demo_sk#57] -Join type: Inner -Join condition: None +(70) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] +Right output [2]: [hd_demo_sk#52, hd_income_band_sk#53] +Arguments: [ss_hdemo_sk#4], [hd_demo_sk#52], Inner, BuildRight -(78) Project [codegen id : 24] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_demo_sk#57, hd_income_band_sk#58] +(71) CometProject +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_demo_sk#52, hd_income_band_sk#53] +Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53] -(79) ReusedExchange [Reuses operator id: 76] -Output [2]: [hd_demo_sk#59, hd_income_band_sk#60] +(72) ReusedExchange [Reuses operator id: 69] +Output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -(80) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [c_current_hdemo_sk#44] -Right keys [1]: [hd_demo_sk#59] -Join type: Inner -Join condition: None +(73) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53] +Right output [2]: [hd_demo_sk#54, hd_income_band_sk#55] +Arguments: [c_current_hdemo_sk#39], [hd_demo_sk#54], Inner, BuildRight -(81) Project [codegen id : 24] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60] -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_demo_sk#59, hd_income_band_sk#60] +(74) CometProject +Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_demo_sk#54, hd_income_band_sk#55] +Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55] -(82) Scan parquet spark_catalog.default.customer_address -Output [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +(75) Scan parquet spark_catalog.default.customer_address +Output [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(83) CometFilter -Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -Condition : isnotnull(ca_address_sk#61) +(76) CometFilter +Input [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +Condition : isnotnull(ca_address_sk#56) -(84) ColumnarToRow [codegen id : 19] -Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +(77) CometBroadcastExchange +Input [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +Arguments: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] -(85) BroadcastExchange -Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=11] +(78) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55] +Right output [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +Arguments: [ss_addr_sk#5], [ca_address_sk#56], Inner, BuildRight -(86) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_addr_sk#5] -Right keys [1]: [ca_address_sk#61] -Join type: Inner -Join condition: None - -(87) Project [codegen id : 24] -Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +(79) CometProject +Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] -(88) ReusedExchange [Reuses operator id: 85] -Output [5]: [ca_address_sk#66, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] +(80) ReusedExchange [Reuses operator id: 77] +Output [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -(89) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [c_current_addr_sk#45] -Right keys [1]: [ca_address_sk#66] -Join type: Inner -Join condition: None +(81) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +Right output [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Arguments: [c_current_addr_sk#40], [ca_address_sk#61], Inner, BuildRight -(90) Project [codegen id : 24] -Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_address_sk#66, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] +(82) CometProject +Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -(91) Scan parquet spark_catalog.default.income_band -Output [1]: [ib_income_band_sk#71] +(83) Scan parquet spark_catalog.default.income_band +Output [1]: [ib_income_band_sk#66] Batched: true Location [not included in comparison]/{warehouse_dir}/income_band] PushedFilters: [IsNotNull(ib_income_band_sk)] ReadSchema: struct -(92) CometFilter -Input [1]: [ib_income_band_sk#71] -Condition : isnotnull(ib_income_band_sk#71) +(84) CometFilter +Input [1]: [ib_income_band_sk#66] +Condition : isnotnull(ib_income_band_sk#66) -(93) ColumnarToRow [codegen id : 21] -Input [1]: [ib_income_band_sk#71] +(85) CometBroadcastExchange +Input [1]: [ib_income_band_sk#66] +Arguments: [ib_income_band_sk#66] -(94) BroadcastExchange -Input [1]: [ib_income_band_sk#71] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] - -(95) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [hd_income_band_sk#58] -Right keys [1]: [ib_income_band_sk#71] -Join type: Inner -Join condition: None +(86) CometBroadcastHashJoin +Left output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Right output [1]: [ib_income_band_sk#66] +Arguments: [hd_income_band_sk#53], [ib_income_band_sk#66], Inner, BuildRight -(96) Project [codegen id : 24] -Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, ib_income_band_sk#71] +(87) CometProject +Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ib_income_band_sk#66] +Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -(97) ReusedExchange [Reuses operator id: 94] -Output [1]: [ib_income_band_sk#72] +(88) ReusedExchange [Reuses operator id: 85] +Output [1]: [ib_income_band_sk#67] -(98) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [hd_income_band_sk#60] -Right keys [1]: [ib_income_band_sk#72] -Join type: Inner -Join condition: None +(89) CometBroadcastHashJoin +Left output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Right output [1]: [ib_income_band_sk#67] +Arguments: [hd_income_band_sk#55], [ib_income_band_sk#67], Inner, BuildRight -(99) Project [codegen id : 24] -Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, ib_income_band_sk#72] +(90) CometProject +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ib_income_band_sk#67] +Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -(100) Scan parquet spark_catalog.default.item -Output [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] +(91) Scan parquet spark_catalog.default.item +Output [4]: [i_item_sk#68, i_current_price#69, i_color#70, i_product_name#71] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), In(i_color, [burlywood ,floral ,indian ,medium ,purple ,spring ]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] ReadSchema: struct -(101) CometFilter -Input [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] -Condition : ((((((isnotnull(i_current_price#74) AND i_color#75 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#74 >= 64.00)) AND (i_current_price#74 <= 74.00)) AND (i_current_price#74 >= 65.00)) AND (i_current_price#74 <= 79.00)) AND isnotnull(i_item_sk#73)) - -(102) CometProject -Input [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] -Arguments: [i_item_sk#73, i_product_name#76], [i_item_sk#73, i_product_name#76] +(92) CometFilter +Input [4]: [i_item_sk#68, i_current_price#69, i_color#70, i_product_name#71] +Condition : ((((((isnotnull(i_current_price#69) AND i_color#70 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#69 >= 64.00)) AND (i_current_price#69 <= 74.00)) AND (i_current_price#69 >= 65.00)) AND (i_current_price#69 <= 79.00)) AND isnotnull(i_item_sk#68)) -(103) ColumnarToRow [codegen id : 23] -Input [2]: [i_item_sk#73, i_product_name#76] +(93) CometProject +Input [4]: [i_item_sk#68, i_current_price#69, i_color#70, i_product_name#71] +Arguments: [i_item_sk#68, i_product_name#71], [i_item_sk#68, i_product_name#71] -(104) BroadcastExchange -Input [2]: [i_item_sk#73, i_product_name#76] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] +(94) CometBroadcastExchange +Input [2]: [i_item_sk#68, i_product_name#71] +Arguments: [i_item_sk#68, i_product_name#71] -(105) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#73] -Join type: Inner -Join condition: None +(95) CometBroadcastHashJoin +Left output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Right output [2]: [i_item_sk#68, i_product_name#71] +Arguments: [ss_item_sk#1], [i_item_sk#68], Inner, BuildRight -(106) Project [codegen id : 24] -Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, d_year#49, d_year#51, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] +(96) CometProject +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, i_item_sk#68, i_product_name#71] +Arguments: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#44, d_year#46, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, i_item_sk#68, i_product_name#71], [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#44, d_year#46, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, i_item_sk#68, i_product_name#71] -(107) HashAggregate [codegen id : 24] -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, d_year#49, d_year#51, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51] +(97) CometHashAggregate +Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#44, d_year#46, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, i_item_sk#68, i_product_name#71] +Keys [15]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46] Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count#77, sum#78, sum#79, sum#80] -Results [19]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51, count#81, sum#82, sum#83, sum#84] -(108) HashAggregate [codegen id : 24] -Input [19]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51, count#81, sum#82, sum#83, sum#84] -Keys [15]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51] +(98) CometHashAggregate +Input [19]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46, count#72, sum#73, sum#74, sum#75] +Keys [15]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46] Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count(1)#85, sum(UnscaledValue(ss_wholesale_cost#9))#86, sum(UnscaledValue(ss_list_price#10))#87, sum(UnscaledValue(ss_coupon_amt#11))#88] -Results [17]: [i_product_name#76 AS product_name#89, i_item_sk#73 AS item_sk#90, s_store_name#40 AS store_name#91, s_zip#41 AS store_zip#92, ca_street_number#62 AS b_street_number#93, ca_street_name#63 AS b_streen_name#94, ca_city#64 AS b_city#95, ca_zip#65 AS b_zip#96, ca_street_number#67 AS c_street_number#97, ca_street_name#68 AS c_street_name#98, ca_city#69 AS c_city#99, ca_zip#70 AS c_zip#100, d_year#38 AS syear#101, count(1)#85 AS cnt#102, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#86,17,2) AS s1#103, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#87,17,2) AS s2#104, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#88,17,2) AS s3#105] -(109) Exchange -Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] -Arguments: hashpartitioning(item_sk#90, store_name#91, store_zip#92, 5), ENSURE_REQUIREMENTS, [plan_id=14] +(99) CometColumnarExchange +Input [17]: [product_name#76, item_sk#77, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92] +Arguments: hashpartitioning(item_sk#77, store_name#78, store_zip#79, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(100) CometSort +Input [17]: [product_name#76, item_sk#77, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92] +Arguments: [product_name#76, item_sk#77, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92], [item_sk#77 ASC NULLS FIRST, store_name#78 ASC NULLS FIRST, store_zip#79 ASC NULLS FIRST] -(110) Sort [codegen id : 25] -Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] -Arguments: [item_sk#90 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, store_zip#92 ASC NULLS FIRST], false, 0 +(101) ColumnarToRow [codegen id : 1] +Input [17]: [product_name#76, item_sk#77, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92] -(111) Scan parquet spark_catalog.default.store_sales -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] +(102) Scan parquet spark_catalog.default.store_sales +Output [12]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_ticket_number#100, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#117), dynamicpruningexpression(ss_sold_date_sk#117 IN dynamicpruning#118)] +PartitionFilters: [isnotnull(ss_sold_date_sk#104), dynamicpruningexpression(ss_sold_date_sk#104 IN dynamicpruning#105)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct -(112) CometFilter -Input [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] -Condition : (((((((isnotnull(ss_item_sk#106) AND isnotnull(ss_ticket_number#113)) AND isnotnull(ss_store_sk#111)) AND isnotnull(ss_customer_sk#107)) AND isnotnull(ss_cdemo_sk#108)) AND isnotnull(ss_promo_sk#112)) AND isnotnull(ss_hdemo_sk#109)) AND isnotnull(ss_addr_sk#110)) +(103) CometFilter +Input [12]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_ticket_number#100, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] +Condition : (((((((isnotnull(ss_item_sk#93) AND isnotnull(ss_ticket_number#100)) AND isnotnull(ss_store_sk#98)) AND isnotnull(ss_customer_sk#94)) AND isnotnull(ss_cdemo_sk#95)) AND isnotnull(ss_promo_sk#99)) AND isnotnull(ss_hdemo_sk#96)) AND isnotnull(ss_addr_sk#97)) -(113) CometBroadcastExchange -Input [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] -Arguments: [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] +(104) CometBroadcastExchange +Input [12]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_ticket_number#100, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] +Arguments: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_ticket_number#100, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] -(114) Scan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] +(105) Scan parquet spark_catalog.default.store_returns +Output [3]: [sr_item_sk#106, sr_ticket_number#107, sr_returned_date_sk#108] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] ReadSchema: struct -(115) CometFilter -Input [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] -Condition : (isnotnull(sr_item_sk#119) AND isnotnull(sr_ticket_number#120)) +(106) CometFilter +Input [3]: [sr_item_sk#106, sr_ticket_number#107, sr_returned_date_sk#108] +Condition : (isnotnull(sr_item_sk#106) AND isnotnull(sr_ticket_number#107)) -(116) CometProject -Input [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] -Arguments: [sr_item_sk#119, sr_ticket_number#120], [sr_item_sk#119, sr_ticket_number#120] +(107) CometProject +Input [3]: [sr_item_sk#106, sr_ticket_number#107, sr_returned_date_sk#108] +Arguments: [sr_item_sk#106, sr_ticket_number#107], [sr_item_sk#106, sr_ticket_number#107] -(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, BuildLeft +(108) CometBroadcastHashJoin +Left output [12]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_ticket_number#100, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] +Right output [2]: [sr_item_sk#106, sr_ticket_number#107] +Arguments: [ss_item_sk#93, ss_ticket_number#100], [sr_item_sk#106, sr_ticket_number#107], 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] -Arguments: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117], [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +(109) CometProject +Input [14]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_ticket_number#100, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104, sr_item_sk#106, sr_ticket_number#107] +Arguments: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104], [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] -(119) ColumnarToRow [codegen id : 26] -Input [11]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +(110) CometColumnarExchange +Input [11]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] +Arguments: hashpartitioning(ss_item_sk#93, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(120) Exchange -Input [11]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -Arguments: hashpartitioning(ss_item_sk#106, 5), ENSURE_REQUIREMENTS, [plan_id=15] +(111) CometSort +Input [11]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] +Arguments: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104], [ss_item_sk#93 ASC NULLS FIRST] -(121) Sort [codegen id : 27] -Input [11]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -Arguments: [ss_item_sk#106 ASC NULLS FIRST], false, 0 +(112) ReusedExchange [Reuses operator id: 24] +Output [4]: [cs_item_sk#109, sum#110, sum#111, isEmpty#112] -(122) ReusedExchange [Reuses operator id: 27] -Output [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] +(113) CometHashAggregate +Input [4]: [cs_item_sk#109, sum#110, sum#111, isEmpty#112] +Keys [1]: [cs_item_sk#109] +Functions [2]: [sum(UnscaledValue(cs_ext_list_price#113)), sum(((cr_refunded_cash#114 + cr_reversed_charge#115) + cr_store_credit#116))] -(123) HashAggregate [codegen id : 33] -Input [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] -Keys [1]: [cs_item_sk#122] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#126)), sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#126))#33, sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))#34] -Results [3]: [cs_item_sk#122, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#126))#33,17,2) AS sale#35, sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))#34 AS refund#36] +(114) CometFilter +Input [3]: [cs_item_sk#109, sale#30, refund#31] +Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) -(124) Filter [codegen id : 33] -Input [3]: [cs_item_sk#122, sale#35, refund#36] -Condition : ((isnotnull(sale#35) AND isnotnull(refund#36)) AND (cast(sale#35 as decimal(21,2)) > (2 * refund#36))) +(115) CometProject +Input [3]: [cs_item_sk#109, sale#30, refund#31] +Arguments: [cs_item_sk#109], [cs_item_sk#109] -(125) Project [codegen id : 33] -Output [1]: [cs_item_sk#122] -Input [3]: [cs_item_sk#122, sale#35, refund#36] +(116) CometSort +Input [1]: [cs_item_sk#109] +Arguments: [cs_item_sk#109], [cs_item_sk#109 ASC NULLS FIRST] -(126) Sort [codegen id : 33] -Input [1]: [cs_item_sk#122] -Arguments: [cs_item_sk#122 ASC NULLS FIRST], false, 0 +(117) CometSortMergeJoin +Left output [11]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] +Right output [1]: [cs_item_sk#109] +Arguments: [ss_item_sk#93], [cs_item_sk#109], Inner -(127) SortMergeJoin [codegen id : 49] -Left keys [1]: [ss_item_sk#106] -Right keys [1]: [cs_item_sk#122] -Join type: Inner -Join condition: None +(118) CometProject +Input [12]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104, cs_item_sk#109] +Arguments: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104], [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] -(128) Project [codegen id : 49] -Output [11]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -Input [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, cs_item_sk#122] +(119) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#117, d_year#118] +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 -(129) ReusedExchange [Reuses operator id: 189] -Output [2]: [d_date_sk#130, d_year#131] +(120) CometFilter +Input [2]: [d_date_sk#117, d_year#118] +Condition : ((isnotnull(d_year#118) AND (d_year#118 = 2000)) AND isnotnull(d_date_sk#117)) -(130) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [ss_sold_date_sk#117] -Right keys [1]: [d_date_sk#130] -Join type: Inner -Join condition: None +(121) CometBroadcastExchange +Input [2]: [d_date_sk#117, d_year#118] +Arguments: [d_date_sk#117, d_year#118] -(131) Project [codegen id : 49] -Output [11]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131] -Input [13]: [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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, d_date_sk#130, d_year#131] +(122) CometBroadcastHashJoin +Left output [11]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104] +Right output [2]: [d_date_sk#117, d_year#118] +Arguments: [ss_sold_date_sk#104], [d_date_sk#117], Inner, BuildRight -(132) ReusedExchange [Reuses operator id: 40] -Output [3]: [s_store_sk#132, s_store_name#133, s_zip#134] +(123) CometProject +Input [13]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, ss_sold_date_sk#104, d_date_sk#117, d_year#118] +Arguments: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118], [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118] -(133) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [ss_store_sk#111] -Right keys [1]: [s_store_sk#132] -Join type: Inner -Join condition: None +(124) ReusedExchange [Reuses operator id: 38] +Output [3]: [s_store_sk#119, s_store_name#120, s_zip#121] -(134) Project [codegen id : 49] -Output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134] -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_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_sk#132, s_store_name#133, s_zip#134] +(125) CometBroadcastHashJoin +Left output [11]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118] +Right output [3]: [s_store_sk#119, s_store_name#120, s_zip#121] +Arguments: [ss_store_sk#98], [s_store_sk#119], Inner, BuildRight -(135) ReusedExchange [Reuses operator id: 46] -Output [6]: [c_customer_sk#135, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, c_first_sales_date_sk#140] +(126) CometProject +Input [14]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_store_sk#98, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_sk#119, s_store_name#120, s_zip#121] +Arguments: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121], [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121] -(136) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [ss_customer_sk#107] -Right keys [1]: [c_customer_sk#135] -Join type: Inner -Join condition: None +(127) ReusedExchange [Reuses operator id: 43] +Output [6]: [c_customer_sk#122, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, c_first_sales_date_sk#127] -(137) Project [codegen id : 49] -Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, c_first_sales_date_sk#140] -Input [18]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_customer_sk#135, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, c_first_sales_date_sk#140] +(128) CometBroadcastHashJoin +Left output [12]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121] +Right output [6]: [c_customer_sk#122, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, c_first_sales_date_sk#127] +Arguments: [ss_customer_sk#94], [c_customer_sk#122], Inner, BuildRight -(138) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#141, d_year#142] +(129) CometProject +Input [18]: [ss_item_sk#93, ss_customer_sk#94, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_customer_sk#122, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, c_first_sales_date_sk#127] +Arguments: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, c_first_sales_date_sk#127], [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, c_first_sales_date_sk#127] -(139) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [c_first_sales_date_sk#140] -Right keys [1]: [d_date_sk#141] -Join type: Inner -Join condition: None +(130) ReusedExchange [Reuses operator id: 48] +Output [2]: [d_date_sk#128, d_year#129] -(140) Project [codegen id : 49] -Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, d_year#142] -Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, c_first_sales_date_sk#140, d_date_sk#141, d_year#142] +(131) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, c_first_sales_date_sk#127] +Right output [2]: [d_date_sk#128, d_year#129] +Arguments: [c_first_sales_date_sk#127], [d_date_sk#128], Inner, BuildRight -(141) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#143, d_year#144] +(132) CometProject +Input [18]: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, c_first_sales_date_sk#127, d_date_sk#128, d_year#129] +Arguments: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, d_year#129], [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, d_year#129] -(142) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [c_first_shipto_date_sk#139] -Right keys [1]: [d_date_sk#143] -Join type: Inner -Join condition: None +(133) ReusedExchange [Reuses operator id: 48] +Output [2]: [d_date_sk#130, d_year#131] -(143) Project [codegen id : 49] -Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144] -Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, d_year#142, d_date_sk#143, d_year#144] +(134) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, d_year#129] +Right output [2]: [d_date_sk#130, d_year#131] +Arguments: [c_first_shipto_date_sk#126], [d_date_sk#130], Inner, BuildRight -(144) ReusedExchange [Reuses operator id: 61] -Output [2]: [cd_demo_sk#145, cd_marital_status#146] +(135) CometProject +Input [18]: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, c_first_shipto_date_sk#126, d_year#129, d_date_sk#130, d_year#131] +Arguments: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131], [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131] -(145) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [ss_cdemo_sk#108] -Right keys [1]: [cd_demo_sk#145] -Join type: Inner -Join condition: None +(136) ReusedExchange [Reuses operator id: 56] +Output [2]: [cd_demo_sk#132, cd_marital_status#133] -(146) Project [codegen id : 49] -Output [16]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, cd_marital_status#146] -Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, cd_demo_sk#145, cd_marital_status#146] +(137) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131] +Right output [2]: [cd_demo_sk#132, cd_marital_status#133] +Arguments: [ss_cdemo_sk#95], [cd_demo_sk#132], Inner, BuildRight -(147) ReusedExchange [Reuses operator id: 61] -Output [2]: [cd_demo_sk#147, cd_marital_status#148] +(138) CometProject +Input [18]: [ss_item_sk#93, ss_cdemo_sk#95, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, cd_demo_sk#132, cd_marital_status#133] +Arguments: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, cd_marital_status#133], [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, cd_marital_status#133] -(148) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [c_current_cdemo_sk#136] -Right keys [1]: [cd_demo_sk#147] -Join type: Inner -Join condition: NOT (cd_marital_status#146 = cd_marital_status#148) +(139) ReusedExchange [Reuses operator id: 56] +Output [2]: [cd_demo_sk#134, cd_marital_status#135] -(149) Project [codegen id : 49] -Output [14]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144] -Input [18]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, cd_marital_status#146, cd_demo_sk#147, cd_marital_status#148] +(140) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, cd_marital_status#133] +Right output [2]: [cd_demo_sk#134, cd_marital_status#135] +Arguments: [c_current_cdemo_sk#123], [cd_demo_sk#134], Inner, NOT (cd_marital_status#133 = cd_marital_status#135), BuildRight -(150) ReusedExchange [Reuses operator id: 70] -Output [1]: [p_promo_sk#149] +(141) CometProject +Input [18]: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_cdemo_sk#123, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, cd_marital_status#133, cd_demo_sk#134, cd_marital_status#135] +Arguments: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131], [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131] -(151) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [ss_promo_sk#112] -Right keys [1]: [p_promo_sk#149] -Join type: Inner -Join condition: None +(142) ReusedExchange [Reuses operator id: 64] +Output [1]: [p_promo_sk#136] -(152) Project [codegen id : 49] -Output [13]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144] -Input [15]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, p_promo_sk#149] +(143) CometBroadcastHashJoin +Left output [14]: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131] +Right output [1]: [p_promo_sk#136] +Arguments: [ss_promo_sk#99], [p_promo_sk#136], Inner, BuildRight -(153) ReusedExchange [Reuses operator id: 76] -Output [2]: [hd_demo_sk#150, hd_income_band_sk#151] +(144) CometProject +Input [15]: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_promo_sk#99, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, p_promo_sk#136] +Arguments: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131], [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131] -(154) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [ss_hdemo_sk#109] -Right keys [1]: [hd_demo_sk#150] -Join type: Inner -Join condition: None +(145) ReusedExchange [Reuses operator id: 69] +Output [2]: [hd_demo_sk#137, hd_income_band_sk#138] -(155) Project [codegen id : 49] -Output [13]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151] -Input [15]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, hd_demo_sk#150, hd_income_band_sk#151] +(146) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131] +Right output [2]: [hd_demo_sk#137, hd_income_band_sk#138] +Arguments: [ss_hdemo_sk#96], [hd_demo_sk#137], Inner, BuildRight -(156) ReusedExchange [Reuses operator id: 76] -Output [2]: [hd_demo_sk#152, hd_income_band_sk#153] +(147) CometProject +Input [15]: [ss_item_sk#93, ss_hdemo_sk#96, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, hd_demo_sk#137, hd_income_band_sk#138] +Arguments: [ss_item_sk#93, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138], [ss_item_sk#93, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138] -(157) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [c_current_hdemo_sk#137] -Right keys [1]: [hd_demo_sk#152] -Join type: Inner -Join condition: None +(148) ReusedExchange [Reuses operator id: 69] +Output [2]: [hd_demo_sk#139, hd_income_band_sk#140] -(158) Project [codegen id : 49] -Output [13]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153] -Input [15]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_demo_sk#152, hd_income_band_sk#153] +(149) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#93, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138] +Right output [2]: [hd_demo_sk#139, hd_income_band_sk#140] +Arguments: [c_current_hdemo_sk#124], [hd_demo_sk#139], Inner, BuildRight -(159) ReusedExchange [Reuses operator id: 85] -Output [5]: [ca_address_sk#154, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] +(150) CometProject +Input [15]: [ss_item_sk#93, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_hdemo_sk#124, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_demo_sk#139, hd_income_band_sk#140] +Arguments: [ss_item_sk#93, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140], [ss_item_sk#93, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140] -(160) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [ss_addr_sk#110] -Right keys [1]: [ca_address_sk#154] -Join type: Inner -Join condition: None +(151) ReusedExchange [Reuses operator id: 77] +Output [5]: [ca_address_sk#141, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] -(161) Project [codegen id : 49] -Output [16]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] -Input [18]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_address_sk#154, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] +(152) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#93, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140] +Right output [5]: [ca_address_sk#141, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Arguments: [ss_addr_sk#97], [ca_address_sk#141], Inner, BuildRight -(162) ReusedExchange [Reuses operator id: 85] -Output [5]: [ca_address_sk#159, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] +(153) CometProject +Input [18]: [ss_item_sk#93, ss_addr_sk#97, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_address_sk#141, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Arguments: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145], [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] -(163) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [c_current_addr_sk#138] -Right keys [1]: [ca_address_sk#159] -Join type: Inner -Join condition: None +(154) ReusedExchange [Reuses operator id: 77] +Output [5]: [ca_address_sk#146, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] -(164) Project [codegen id : 49] -Output [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] -Input [21]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_address_sk#159, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] +(155) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Right output [5]: [ca_address_sk#146, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] +Arguments: [c_current_addr_sk#125], [ca_address_sk#146], Inner, BuildRight -(165) ReusedExchange [Reuses operator id: 94] -Output [1]: [ib_income_band_sk#164] +(156) CometProject +Input [21]: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, c_current_addr_sk#125, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_address_sk#146, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] +Arguments: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150], [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] -(166) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [hd_income_band_sk#151] -Right keys [1]: [ib_income_band_sk#164] -Join type: Inner -Join condition: None +(157) ReusedExchange [Reuses operator id: 85] +Output [1]: [ib_income_band_sk#151] -(167) Project [codegen id : 49] -Output [18]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] -Input [20]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, ib_income_band_sk#164] +(158) CometBroadcastHashJoin +Left output [19]: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] +Right output [1]: [ib_income_band_sk#151] +Arguments: [hd_income_band_sk#138], [ib_income_band_sk#151], Inner, BuildRight -(168) ReusedExchange [Reuses operator id: 94] -Output [1]: [ib_income_band_sk#165] +(159) CometProject +Input [20]: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, hd_income_band_sk#138, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, ib_income_band_sk#151] +Arguments: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150], [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] -(169) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [hd_income_band_sk#153] -Right keys [1]: [ib_income_band_sk#165] -Join type: Inner -Join condition: None +(160) ReusedExchange [Reuses operator id: 85] +Output [1]: [ib_income_band_sk#152] -(170) Project [codegen id : 49] -Output [17]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] -Input [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, ib_income_band_sk#165] +(161) CometBroadcastHashJoin +Left output [18]: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] +Right output [1]: [ib_income_band_sk#152] +Arguments: [hd_income_band_sk#140], [ib_income_band_sk#152], Inner, BuildRight -(171) ReusedExchange [Reuses operator id: 104] -Output [2]: [i_item_sk#166, i_product_name#167] +(162) CometProject +Input [19]: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, hd_income_band_sk#140, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, ib_income_band_sk#152] +Arguments: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150], [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] -(172) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [ss_item_sk#106] -Right keys [1]: [i_item_sk#166] -Join type: Inner -Join condition: None - -(173) Project [codegen id : 49] -Output [18]: [ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, d_year#142, d_year#144, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, i_item_sk#166, i_product_name#167] -Input [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, i_item_sk#166, i_product_name#167] - -(174) HashAggregate [codegen id : 49] -Input [18]: [ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, d_year#142, d_year#144, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, i_item_sk#166, i_product_name#167] -Keys [15]: [i_product_name#167, i_item_sk#166, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, d_year#131, d_year#142, d_year#144] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#114)), partial_sum(UnscaledValue(ss_list_price#115)), partial_sum(UnscaledValue(ss_coupon_amt#116))] -Aggregate Attributes [4]: [count#77, sum#168, sum#169, sum#170] -Results [19]: [i_product_name#167, i_item_sk#166, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, d_year#131, d_year#142, d_year#144, count#81, sum#171, sum#172, sum#173] - -(175) HashAggregate [codegen id : 49] -Input [19]: [i_product_name#167, i_item_sk#166, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, d_year#131, d_year#142, d_year#144, count#81, sum#171, sum#172, sum#173] -Keys [15]: [i_product_name#167, i_item_sk#166, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, d_year#131, d_year#142, d_year#144] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#114)), sum(UnscaledValue(ss_list_price#115)), sum(UnscaledValue(ss_coupon_amt#116))] -Aggregate Attributes [4]: [count(1)#85, sum(UnscaledValue(ss_wholesale_cost#114))#86, sum(UnscaledValue(ss_list_price#115))#87, sum(UnscaledValue(ss_coupon_amt#116))#88] -Results [8]: [i_item_sk#166 AS item_sk#174, s_store_name#133 AS store_name#175, s_zip#134 AS store_zip#176, d_year#131 AS syear#177, count(1)#85 AS cnt#178, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#114))#86,17,2) AS s1#179, MakeDecimal(sum(UnscaledValue(ss_list_price#115))#87,17,2) AS s2#180, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#116))#88,17,2) AS s3#181] - -(176) Exchange -Input [8]: [item_sk#174, store_name#175, store_zip#176, syear#177, cnt#178, s1#179, s2#180, s3#181] -Arguments: hashpartitioning(item_sk#174, store_name#175, store_zip#176, 5), ENSURE_REQUIREMENTS, [plan_id=16] - -(177) Sort [codegen id : 50] -Input [8]: [item_sk#174, store_name#175, store_zip#176, syear#177, cnt#178, s1#179, s2#180, s3#181] -Arguments: [item_sk#174 ASC NULLS FIRST, store_name#175 ASC NULLS FIRST, store_zip#176 ASC NULLS FIRST], false, 0 - -(178) SortMergeJoin [codegen id : 51] -Left keys [3]: [item_sk#90, store_name#91, store_zip#92] -Right keys [3]: [item_sk#174, store_name#175, store_zip#176] +(163) ReusedExchange [Reuses operator id: 94] +Output [2]: [i_item_sk#153, i_product_name#154] + +(164) CometBroadcastHashJoin +Left output [17]: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150] +Right output [2]: [i_item_sk#153, i_product_name#154] +Arguments: [ss_item_sk#93], [i_item_sk#153], Inner, BuildRight + +(165) CometProject +Input [19]: [ss_item_sk#93, ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, s_store_name#120, s_zip#121, d_year#129, d_year#131, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, i_item_sk#153, i_product_name#154] +Arguments: [ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, d_year#129, d_year#131, s_store_name#120, s_zip#121, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, i_item_sk#153, i_product_name#154], [ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, d_year#129, d_year#131, s_store_name#120, s_zip#121, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, i_item_sk#153, i_product_name#154] + +(166) CometHashAggregate +Input [18]: [ss_wholesale_cost#101, ss_list_price#102, ss_coupon_amt#103, d_year#118, d_year#129, d_year#131, s_store_name#120, s_zip#121, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, i_item_sk#153, i_product_name#154] +Keys [15]: [i_product_name#154, i_item_sk#153, s_store_name#120, s_zip#121, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, d_year#118, d_year#129, d_year#131] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#101)), partial_sum(UnscaledValue(ss_list_price#102)), partial_sum(UnscaledValue(ss_coupon_amt#103))] + +(167) CometHashAggregate +Input [19]: [i_product_name#154, i_item_sk#153, s_store_name#120, s_zip#121, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, d_year#118, d_year#129, d_year#131, count#72, sum#155, sum#156, sum#157] +Keys [15]: [i_product_name#154, i_item_sk#153, s_store_name#120, s_zip#121, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ca_street_number#147, ca_street_name#148, ca_city#149, ca_zip#150, d_year#118, d_year#129, d_year#131] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#101)), sum(UnscaledValue(ss_list_price#102)), sum(UnscaledValue(ss_coupon_amt#103))] + +(168) CometColumnarExchange +Input [8]: [item_sk#158, store_name#159, store_zip#160, syear#161, cnt#162, s1#163, s2#164, s3#165] +Arguments: hashpartitioning(item_sk#158, store_name#159, store_zip#160, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(169) CometSort +Input [8]: [item_sk#158, store_name#159, store_zip#160, syear#161, cnt#162, s1#163, s2#164, s3#165] +Arguments: [item_sk#158, store_name#159, store_zip#160, syear#161, cnt#162, s1#163, s2#164, s3#165], [item_sk#158 ASC NULLS FIRST, store_name#159 ASC NULLS FIRST, store_zip#160 ASC NULLS FIRST] + +(170) ColumnarToRow [codegen id : 2] +Input [8]: [item_sk#158, store_name#159, store_zip#160, syear#161, cnt#162, s1#163, s2#164, s3#165] + +(171) SortMergeJoin [codegen id : 3] +Left keys [3]: [item_sk#77, store_name#78, store_zip#79] +Right keys [3]: [item_sk#158, store_name#159, store_zip#160] Join type: Inner -Join condition: (cnt#178 <= cnt#102) +Join condition: (cnt#162 <= cnt#89) -(179) Project [codegen id : 51] -Output [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#179, s2#180, s3#181, syear#177, cnt#178] -Input [25]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, item_sk#174, store_name#175, store_zip#176, syear#177, cnt#178, s1#179, s2#180, s3#181] +(172) Project [codegen id : 3] +Output [21]: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162] +Input [25]: [product_name#76, item_sk#77, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, item_sk#158, store_name#159, store_zip#160, syear#161, cnt#162, s1#163, s2#164, s3#165] -(180) Exchange -Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#179, s2#180, s3#181, syear#177, cnt#178] -Arguments: rangepartitioning(product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#178 ASC NULLS FIRST, s1#103 ASC NULLS FIRST, s1#179 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=17] +(173) RowToColumnar +Input [21]: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162] -(181) Sort [codegen id : 52] -Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#179, s2#180, s3#181, syear#177, cnt#178] -Arguments: [product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#178 ASC NULLS FIRST, s1#103 ASC NULLS FIRST, s1#179 ASC NULLS FIRST], true, 0 +(174) CometColumnarExchange +Input [21]: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162] +Arguments: rangepartitioning(product_name#76 ASC NULLS FIRST, store_name#78 ASC NULLS FIRST, cnt#162 ASC NULLS FIRST, s1#90 ASC NULLS FIRST, s1#163 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(175) CometSort +Input [21]: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162] +Arguments: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162], [product_name#76 ASC NULLS FIRST, store_name#78 ASC NULLS FIRST, cnt#162 ASC NULLS FIRST, s1#90 ASC NULLS FIRST, s1#163 ASC NULLS FIRST] + +(176) ColumnarToRow [codegen id : 4] +Input [21]: [product_name#76, store_name#78, store_zip#79, b_street_number#80, b_streen_name#81, b_city#82, b_zip#83, c_street_number#84, c_street_name#85, c_city#86, c_zip#87, syear#88, cnt#89, s1#90, s2#91, s3#92, s1#163, s2#164, s3#165, syear#161, cnt#162] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (185) -+- * ColumnarToRow (184) - +- CometFilter (183) - +- CometScan parquet spark_catalog.default.date_dim (182) +BroadcastExchange (180) ++- * ColumnarToRow (179) + +- CometFilter (178) + +- CometScan parquet spark_catalog.default.date_dim (177) -(182) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#37, d_year#38] +(177) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#32, d_year#33] 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 -(183) CometFilter -Input [2]: [d_date_sk#37, d_year#38] -Condition : ((isnotnull(d_year#38) AND (d_year#38 = 1999)) AND isnotnull(d_date_sk#37)) +(178) CometFilter +Input [2]: [d_date_sk#32, d_year#33] +Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(184) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#37, d_year#38] +(179) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#32, d_year#33] -(185) BroadcastExchange -Input [2]: [d_date_sk#37, d_year#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] +(180) BroadcastExchange +Input [2]: [d_date_sk#32, d_year#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -Subquery:2 Hosting operator id = 111 Hosting Expression = ss_sold_date_sk#117 IN dynamicpruning#118 -BroadcastExchange (189) -+- * ColumnarToRow (188) - +- CometFilter (187) - +- CometScan parquet spark_catalog.default.date_dim (186) +Subquery:2 Hosting operator id = 102 Hosting Expression = ss_sold_date_sk#104 IN dynamicpruning#105 +BroadcastExchange (184) ++- * ColumnarToRow (183) + +- CometFilter (182) + +- CometScan parquet spark_catalog.default.date_dim (181) -(186) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#130, d_year#131] +(181) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#117, d_year#118] 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 -(187) CometFilter -Input [2]: [d_date_sk#130, d_year#131] -Condition : ((isnotnull(d_year#131) AND (d_year#131 = 2000)) AND isnotnull(d_date_sk#130)) +(182) CometFilter +Input [2]: [d_date_sk#117, d_year#118] +Condition : ((isnotnull(d_year#118) AND (d_year#118 = 2000)) AND isnotnull(d_date_sk#117)) -(188) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#130, d_year#131] +(183) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#117, d_year#118] -(189) BroadcastExchange -Input [2]: [d_date_sk#130, d_year#131] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=19] +(184) BroadcastExchange +Input [2]: [d_date_sk#117, d_year#118] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt index 512a74f9f..70d0e66f8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt @@ -1,281 +1,199 @@ -WholeStageCodegen (52) - Sort [product_name,store_name,cnt,s1,s1] +WholeStageCodegen (4) + ColumnarToRow InputAdapter - Exchange [product_name,store_name,cnt,s1,s1] #1 - WholeStageCodegen (51) - Project [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - SortMergeJoin [item_sk,store_name,store_zip,item_sk,store_name,store_zip,cnt,cnt] - InputAdapter - WholeStageCodegen (25) - Sort [item_sk,store_name,store_zip] - InputAdapter - Exchange [item_sk,store_name,store_zip] #2 - WholeStageCodegen (24) - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] - Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SortMergeJoin [ss_item_sk,cs_item_sk] - InputAdapter - WholeStageCodegen (2) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #4 - CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - WholeStageCodegen (8) - Sort [cs_item_sk] - Project [cs_item_sk] - Filter [sale,refund] - HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] - InputAdapter - Exchange [cs_item_sk] #6 - WholeStageCodegen (7) - HashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [sum,sum,isEmpty,sum,sum,isEmpty] - Project [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] - InputAdapter - WholeStageCodegen (4) - Sort [cs_item_sk,cs_order_number] - InputAdapter - Exchange [cs_item_sk,cs_order_number] #7 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - InputAdapter - WholeStageCodegen (6) - Sort [cr_item_sk,cr_order_number] - InputAdapter - Exchange [cr_item_sk,cr_order_number] #8 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_name,s_zip] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (12) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #11 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (14) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk,cd_marital_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometFilter [p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk] - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (17) - 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 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (19) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 - InputAdapter - BroadcastExchange #16 - WholeStageCodegen (21) - ColumnarToRow - InputAdapter - CometFilter [ib_income_band_sk] - CometScan parquet spark_catalog.default.income_band [ib_income_band_sk] - InputAdapter - ReusedExchange [ib_income_band_sk] #16 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (23) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] - InputAdapter - WholeStageCodegen (50) - Sort [item_sk,store_name,store_zip] - InputAdapter - Exchange [item_sk,store_name,store_zip] #18 - WholeStageCodegen (49) - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] - Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SortMergeJoin [ss_item_sk,cs_item_sk] - InputAdapter - WholeStageCodegen (27) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #19 - WholeStageCodegen (26) - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #20 - CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #21 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - WholeStageCodegen (33) - Sort [cs_item_sk] - Project [cs_item_sk] - Filter [sale,refund] - HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] - InputAdapter - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #21 - InputAdapter - ReusedExchange [s_store_sk,s_store_name,s_zip] #9 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #10 - InputAdapter - ReusedExchange [d_date_sk,d_year] #11 - InputAdapter - ReusedExchange [d_date_sk,d_year] #11 - InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 - InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 - InputAdapter - ReusedExchange [p_promo_sk] #13 - InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 - InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 - InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 - InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 - InputAdapter - ReusedExchange [ib_income_band_sk] #16 - InputAdapter - ReusedExchange [ib_income_band_sk] #16 - InputAdapter - ReusedExchange [i_item_sk,i_product_name] #17 + CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] + CometColumnarExchange [product_name,store_name,cnt,s1,s1] #1 + RowToColumnar + WholeStageCodegen (3) + Project [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] + SortMergeJoin [item_sk,store_name,store_zip,item_sk,store_name,store_zip,cnt,cnt] + InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometSort [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3] + CometColumnarExchange [item_sk,store_name,store_zip] #2 + CometHashAggregate [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] + CometHashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum,ss_wholesale_cost,ss_list_price,ss_coupon_amt] + CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] + CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk] #3 + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #4 + CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [cs_item_sk,sale,refund] + CometHashAggregate [cs_item_sk,sale,refund,sum,sum,isEmpty,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] + CometColumnarExchange [cs_item_sk] #6 + CometHashAggregate [cs_item_sk,sum,sum,isEmpty,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometSortMergeJoin [cs_item_sk,cs_order_number,cs_ext_list_price,cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometSort [cs_item_sk,cs_order_number,cs_ext_list_price] + CometColumnarExchange [cs_item_sk,cs_order_number] #7 + CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometColumnarExchange [cr_item_sk,cr_order_number] #8 + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 + CometFilter [s_store_sk,s_store_name,s_zip] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #12 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk,d_year] #12 + CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 + CometFilter [cd_demo_sk,cd_marital_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + ReusedExchange [cd_demo_sk,cd_marital_status] #13 + CometBroadcastExchange [p_promo_sk] #14 + CometFilter [p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk] + CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 + CometFilter [hd_demo_sk,hd_income_band_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] + ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 + CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + CometBroadcastExchange [ib_income_band_sk] #17 + CometFilter [ib_income_band_sk] + CometScan parquet spark_catalog.default.income_band [ib_income_band_sk] + ReusedExchange [ib_income_band_sk] #17 + CometBroadcastExchange [i_item_sk,i_product_name] #18 + CometProject [i_item_sk,i_product_name] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] + InputAdapter + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometSort [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] + CometColumnarExchange [item_sk,store_name,store_zip] #19 + CometHashAggregate [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] + CometHashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum,ss_wholesale_cost,ss_list_price,ss_coupon_amt] + CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] + CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk] #20 + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #21 + CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #22 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [cs_item_sk,sale,refund] + CometHashAggregate [cs_item_sk,sale,refund,sum,sum,isEmpty,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] + ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 + CometBroadcastExchange [d_date_sk,d_year] #23 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [s_store_sk,s_store_name,s_zip] #10 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 + ReusedExchange [d_date_sk,d_year] #12 + ReusedExchange [d_date_sk,d_year] #12 + ReusedExchange [cd_demo_sk,cd_marital_status] #13 + ReusedExchange [cd_demo_sk,cd_marital_status] #13 + ReusedExchange [p_promo_sk] #14 + ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 + ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + ReusedExchange [ib_income_band_sk] #17 + ReusedExchange [ib_income_band_sk] #17 + ReusedExchange [i_item_sk,i_product_name] #18 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 76545b83a..bc2e260ce 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 @@ -2,71 +2,71 @@ 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) + +- * ColumnarToRow (65) + +- CometSort (64) + +- CometColumnarExchange (63) + +- CometUnion (62) + :- CometHashAggregate (21) + : +- CometColumnarExchange (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) + :- CometHashAggregate (26) + : +- CometColumnarExchange (25) + : +- CometHashAggregate (24) + : +- CometHashAggregate (23) + : +- ReusedExchange (22) + :- CometHashAggregate (31) + : +- CometColumnarExchange (30) + : +- CometHashAggregate (29) + : +- CometHashAggregate (28) + : +- ReusedExchange (27) + :- CometHashAggregate (36) + : +- CometColumnarExchange (35) + : +- CometHashAggregate (34) + : +- CometHashAggregate (33) + : +- ReusedExchange (32) + :- CometHashAggregate (41) + : +- CometColumnarExchange (40) + : +- CometHashAggregate (39) + : +- CometHashAggregate (38) + : +- ReusedExchange (37) + :- CometHashAggregate (46) + : +- CometColumnarExchange (45) + : +- CometHashAggregate (44) + : +- CometHashAggregate (43) + : +- ReusedExchange (42) + :- CometHashAggregate (51) + : +- CometColumnarExchange (50) + : +- CometHashAggregate (49) + : +- CometHashAggregate (48) + : +- ReusedExchange (47) + :- CometHashAggregate (56) + : +- CometColumnarExchange (55) + : +- CometHashAggregate (54) + : +- CometHashAggregate (53) + : +- ReusedExchange (52) + +- CometHashAggregate (61) + +- CometColumnarExchange (60) + +- CometHashAggregate (59) + +- CometHashAggregate (58) + +- ReusedExchange (57) (1) Scan parquet spark_catalog.default.store_sales @@ -162,265 +162,224 @@ Input [10]: [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_st 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))] -(20) ColumnarToRow [codegen id : 1] +(20) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(22) HashAggregate [codegen id : 2] +(21) CometHashAggregate 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))#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] -(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] +(22) ReusedExchange [Reuses operator id: 20] +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#21, isEmpty#22] -(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] +(23) CometHashAggregate +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))#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] +(24) CometHashAggregate +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#23] 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] +Functions [1]: [partial_sum(sumsales#23)] -(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] +(25) CometColumnarExchange +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#24, isEmpty#25] +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, CometColumnarShuffle, [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] +(26) CometHashAggregate +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#24, isEmpty#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]: [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] +Functions [1]: [sum(sumsales#23)] -(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] +(27) ReusedExchange [Reuses operator id: 20] +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#26, isEmpty#27] -(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] +(28) CometHashAggregate +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#26, isEmpty#27] 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))#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] +(29) CometHashAggregate +Input [7]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, sumsales#23] 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] +Functions [1]: [partial_sum(sumsales#23)] -(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] +(30) CometColumnarExchange +Input [8]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, 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, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [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] +(31) CometHashAggregate +Input [8]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, sum#28, isEmpty#29] 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] +Functions [1]: [sum(sumsales#23)] -(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] +(32) ReusedExchange [Reuses operator id: 20] +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#30, isEmpty#31] -(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] +(33) CometHashAggregate +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#30, isEmpty#31] 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))#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] +(34) CometHashAggregate +Input [6]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, sumsales#23] 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] +Functions [1]: [partial_sum(sumsales#23)] -(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] +(35) CometColumnarExchange +Input [7]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, sum#32, isEmpty#33] +Arguments: hashpartitioning(i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [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] +(36) CometHashAggregate +Input [7]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, sum#32, isEmpty#33] 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] +Functions [1]: [sum(sumsales#23)] -(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] +(37) ReusedExchange [Reuses operator id: 20] +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#34, isEmpty#35] -(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] +(38) CometHashAggregate +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#34, isEmpty#35] 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))#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] +(39) CometHashAggregate +Input [5]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, sumsales#23] 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] +Functions [1]: [partial_sum(sumsales#23)] -(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] +(40) CometColumnarExchange +Input [6]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, sum#36, isEmpty#37] +Arguments: hashpartitioning(i_category#17, i_class#16, i_brand#15, i_product_name#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [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] +(41) CometHashAggregate +Input [6]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, sum#36, isEmpty#37] 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] +Functions [1]: [sum(sumsales#23)] -(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] +(42) ReusedExchange [Reuses operator id: 20] +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#38, isEmpty#39] -(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] +(43) CometHashAggregate +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#38, isEmpty#39] 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))#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] +(44) CometHashAggregate +Input [4]: [i_category#17, i_class#16, i_brand#15, sumsales#23] 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] +Functions [1]: [partial_sum(sumsales#23)] -(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] +(45) CometColumnarExchange +Input [5]: [i_category#17, i_class#16, i_brand#15, sum#40, isEmpty#41] +Arguments: hashpartitioning(i_category#17, i_class#16, i_brand#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(47) HashAggregate [codegen id : 17] -Input [5]: [i_category#17, i_class#16, i_brand#15, sum#70, isEmpty#71] +(46) CometHashAggregate +Input [5]: [i_category#17, i_class#16, i_brand#15, sum#40, isEmpty#41] 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] +Functions [1]: [sum(sumsales#23)] -(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] +(47) ReusedExchange [Reuses operator id: 20] +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#42, isEmpty#43] -(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] +(48) CometHashAggregate +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#42, isEmpty#43] 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))#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] +(49) CometHashAggregate +Input [3]: [i_category#17, i_class#16, sumsales#23] 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] +Functions [1]: [partial_sum(sumsales#23)] -(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] +(50) CometColumnarExchange +Input [4]: [i_category#17, i_class#16, sum#44, isEmpty#45] +Arguments: hashpartitioning(i_category#17, i_class#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(52) HashAggregate [codegen id : 20] -Input [4]: [i_category#17, i_class#16, sum#83, isEmpty#84] +(51) CometHashAggregate +Input [4]: [i_category#17, i_class#16, sum#44, isEmpty#45] 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] +Functions [1]: [sum(sumsales#23)] -(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] +(52) ReusedExchange [Reuses operator id: 20] +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#46, isEmpty#47] -(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] +(53) CometHashAggregate +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#46, isEmpty#47] 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))#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] +(54) CometHashAggregate +Input [2]: [i_category#17, sumsales#23] 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] +Functions [1]: [partial_sum(sumsales#23)] -(56) Exchange -Input [3]: [i_category#17, sum#97, isEmpty#98] -Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, [plan_id=8] +(55) CometColumnarExchange +Input [3]: [i_category#17, sum#48, isEmpty#49] +Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(57) HashAggregate [codegen id : 23] -Input [3]: [i_category#17, sum#97, isEmpty#98] +(56) CometHashAggregate +Input [3]: [i_category#17, sum#48, isEmpty#49] 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] +Functions [1]: [sum(sumsales#23)] -(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] +(57) ReusedExchange [Reuses operator id: 20] +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#50, isEmpty#51] -(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] +(58) CometHashAggregate +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#50, isEmpty#51] 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))#21] -Results [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#21 AS sumsales#25] -(60) HashAggregate [codegen id : 25] -Input [1]: [sumsales#25] +(59) CometHashAggregate +Input [1]: [sumsales#23] Keys: [] -Functions [1]: [partial_sum(sumsales#25)] -Aggregate Attributes [2]: [sum#110, isEmpty#111] -Results [2]: [sum#112, isEmpty#113] +Functions [1]: [partial_sum(sumsales#23)] -(61) Exchange -Input [2]: [sum#112, isEmpty#113] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] +(60) CometColumnarExchange +Input [2]: [sum#52, isEmpty#53] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(62) HashAggregate [codegen id : 26] -Input [2]: [sum#112, isEmpty#113] +(61) CometHashAggregate +Input [2]: [sum#52, isEmpty#53] Keys: [] -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] - -(63) Union - -(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] - -(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 +Functions [1]: [sum(sumsales#23)] + +(62) CometUnion +Child 0 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#54] +Child 1 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#55, sumsales#56] +Child 2 Input [9]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#57, s_store_id#58, sumsales#59] +Child 3 Input [9]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#60, d_moy#61, s_store_id#62, sumsales#63] +Child 4 Input [9]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#64, d_qoy#65, d_moy#66, s_store_id#67, sumsales#68] +Child 5 Input [9]: [i_category#17, i_class#16, i_brand#15, i_product_name#69, d_year#70, d_qoy#71, d_moy#72, s_store_id#73, sumsales#74] +Child 6 Input [9]: [i_category#17, i_class#16, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80, sumsales#81] +Child 7 Input [9]: [i_category#17, i_class#82, i_brand#83, i_product_name#84, d_year#85, d_qoy#86, d_moy#87, s_store_id#88, sumsales#89] +Child 8 Input [9]: [i_category#90, i_class#91, i_brand#92, i_product_name#93, d_year#94, d_qoy#95, d_moy#96, s_store_id#97, sumsales#98] + +(63) CometColumnarExchange +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#54] +Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(64) CometSort +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#54] +Arguments: [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#54], [i_category#17 ASC NULLS FIRST, sumsales#54 DESC NULLS LAST] + +(65) ColumnarToRow [codegen id : 1] +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#54] (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] +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#54] +Arguments: [rank(sumsales#54) windowspecdefinition(i_category#17, sumsales#54 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#99], [i_category#17], [sumsales#54 DESC NULLS LAST] -(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) Filter [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, sumsales#54, rk#99] +Condition : (rk#99 <= 100) (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] +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#54, rk#99] +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#54 ASC NULLS FIRST, rk#99 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#54, rk#99] ===== Subqueries ===== 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 dfa9ae66d..e631b569c 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,116 +1,80 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (28) + WholeStageCodegen (2) Filter [rk] InputAdapter Window [sumsales,i_category] - WholeStageCodegen (27) - Sort [i_category,sumsales] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [i_category] #1 - Union - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty,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,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] - CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] - CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometFilter [s_store_sk,s_store_id] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - 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] #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 (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] #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 (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] #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 (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] #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 (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] #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 (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] #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 (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] #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 (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 #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 - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometColumnarExchange [i_category] #1 + CometUnion [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty,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,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] + CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty,sum(sumsales)] + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #7 + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty,sumsales] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales,s_store_id,sum,isEmpty,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty,sum(sumsales)] + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #8 + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty,sumsales] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales,d_moy,s_store_id,sum,isEmpty,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty,sum(sumsales)] + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year] #9 + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty,sumsales] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sumsales,d_qoy,d_moy,s_store_id,sum,isEmpty,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty,sum(sumsales)] + CometColumnarExchange [i_category,i_class,i_brand,i_product_name] #10 + CometHashAggregate [i_category,i_class,i_brand,i_product_name,sum,isEmpty,sumsales] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,sumsales,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))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty,sum(sumsales)] + CometColumnarExchange [i_category,i_class,i_brand] #11 + CometHashAggregate [i_category,i_class,i_brand,sum,isEmpty,sumsales] + CometHashAggregate [i_category,i_class,i_brand,sumsales,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))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty,sum(sumsales)] + CometColumnarExchange [i_category,i_class] #12 + CometHashAggregate [i_category,i_class,sum,isEmpty,sumsales] + CometHashAggregate [i_category,i_class,sumsales,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))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty,sum(sumsales)] + CometColumnarExchange [i_category] #13 + CometHashAggregate [i_category,sum,isEmpty,sumsales] + CometHashAggregate [i_category,sumsales,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))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty,sum(sumsales)] + CometColumnarExchange #14 + CometHashAggregate [sum,isEmpty,sumsales] + CometHashAggregate [sumsales,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))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 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 9e148a827..28f77aff2 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,61 +1,71 @@ == Physical Plan == -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) +TakeOrderedAndProject (67) ++- * Project (66) + +- Window (65) + +- * ColumnarToRow (64) + +- CometSort (63) + +- CometColumnarExchange (62) + +- CometHashAggregate (61) + +- CometColumnarExchange (60) + +- RowToColumnar (59) + +- * HashAggregate (58) + +- Union (57) + :- * HashAggregate (40) + : +- * ColumnarToRow (39) + : +- CometColumnarExchange (38) + : +- RowToColumnar (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) + : +- * ColumnarToRow (27) + : +- CometSort (26) + : +- CometHashAggregate (25) + : +- CometColumnarExchange (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 (48) + : +- * ColumnarToRow (47) + : +- CometColumnarExchange (46) + : +- RowToColumnar (45) + : +- * HashAggregate (44) + : +- * HashAggregate (43) + : +- * ColumnarToRow (42) + : +- ReusedExchange (41) + +- * HashAggregate (56) + +- * ColumnarToRow (55) + +- CometColumnarExchange (54) + +- RowToColumnar (53) + +- * HashAggregate (52) + +- * HashAggregate (51) + +- * ColumnarToRow (50) + +- ReusedExchange (49) (1) Scan parquet spark_catalog.default.store_sales @@ -98,7 +108,7 @@ Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight 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] +(9) ColumnarToRow [codegen id : 4] Input [2]: [ss_store_sk#1, ss_net_profit#2] (10) Scan parquet spark_catalog.default.store @@ -112,7 +122,7 @@ ReadSchema: struct Input [3]: [s_store_sk#7, s_county#8, s_state#9] Condition : isnotnull(s_store_sk#7) -(12) ColumnarToRow [codegen id : 4] +(12) ColumnarToRow [codegen id : 3] Input [3]: [s_store_sk#7, s_county#8, s_state#9] (13) Scan parquet spark_catalog.default.store_sales @@ -168,41 +178,39 @@ 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] +(24) CometColumnarExchange Input [2]: [s_state#15, sum#17] +Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(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] +(25) CometHashAggregate 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 +(26) CometSort +Input [3]: [s_state#15, _w0#18, s_state#15] +Arguments: [s_state#15, _w0#18, s_state#15], [s_state#15 ASC NULLS FIRST, _w0#18 DESC NULLS LAST] + +(27) ColumnarToRow [codegen id : 1] +Input [3]: [s_state#15, _w0#18, s_state#15] (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] +Input [3]: [s_state#15, _w0#18, s_state#15] +Arguments: [rank(_w0#18) windowspecdefinition(s_state#15, _w0#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#19], [s_state#15], [_w0#18 DESC NULLS LAST] -(29) Filter [codegen id : 3] -Input [4]: [s_state#15, _w0#19, s_state#15, ranking#20] -Condition : (ranking#20 <= 5) +(29) Filter [codegen id : 2] +Input [4]: [s_state#15, _w0#18, s_state#15, ranking#19] +Condition : (ranking#19 <= 5) -(30) Project [codegen id : 3] +(30) Project [codegen id : 2] Output [1]: [s_state#15] -Input [4]: [s_state#15, _w0#19, s_state#15, ranking#20] +Input [4]: [s_state#15, _w0#18, s_state#15, ranking#19] (31) BroadcastExchange Input [1]: [s_state#15] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] -(32) BroadcastHashJoin [codegen id : 4] +(32) BroadcastHashJoin [codegen id : 3] Left keys [1]: [s_state#9] Right keys [1]: [s_state#15] Join type: LeftSemi @@ -212,159 +220,187 @@ Join condition: None 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] -(34) BroadcastHashJoin [codegen id : 5] +(34) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_store_sk#1] Right keys [1]: [s_store_sk#7] Join type: Inner Join condition: None -(35) Project [codegen id : 5] +(35) Project [codegen id : 4] 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] -(36) HashAggregate [codegen id : 5] +(36) HashAggregate [codegen id : 4] 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#9, s_county#8, sum#22] +Aggregate Attributes [1]: [sum#20] +Results [3]: [s_state#9, s_county#8, sum#21] + +(37) RowToColumnar +Input [3]: [s_state#9, s_county#8, sum#21] -(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] +(38) CometColumnarExchange +Input [3]: [s_state#9, s_county#8, sum#21] +Arguments: hashpartitioning(s_state#9, s_county#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(38) HashAggregate [codegen id : 6] -Input [3]: [s_state#9, s_county#8, sum#22] +(39) ColumnarToRow [codegen id : 5] +Input [3]: [s_state#9, s_county#8, sum#21] + +(40) HashAggregate [codegen id : 5] +Input [3]: [s_state#9, s_county#8, sum#21] 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#9, s_county#8, 0 AS g_state#25, 0 AS g_county#26, 0 AS lochierarchy#27] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#22] +Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#22,17,2) as decimal(27,2)) AS total_sum#23, s_state#9, s_county#8, 0 AS g_state#24, 0 AS g_county#25, 0 AS lochierarchy#26] + +(41) ReusedExchange [Reuses operator id: 38] +Output [3]: [s_state#9, s_county#8, sum#27] -(39) ReusedExchange [Reuses operator id: 37] -Output [3]: [s_state#9, s_county#8, sum#28] +(42) ColumnarToRow [codegen id : 10] +Input [3]: [s_state#9, s_county#8, sum#27] -(40) HashAggregate [codegen id : 12] -Input [3]: [s_state#9, s_county#8, sum#28] +(43) HashAggregate [codegen id : 10] +Input [3]: [s_state#9, s_county#8, sum#27] 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#9] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#22] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#22,17,2) AS total_sum#28, s_state#9] -(41) HashAggregate [codegen id : 12] -Input [2]: [total_sum#29, s_state#9] +(44) HashAggregate [codegen id : 10] +Input [2]: [total_sum#28, 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#9, sum#32, isEmpty#33] +Functions [1]: [partial_sum(total_sum#28)] +Aggregate Attributes [2]: [sum#29, isEmpty#30] +Results [3]: [s_state#9, sum#31, isEmpty#32] + +(45) RowToColumnar +Input [3]: [s_state#9, sum#31, isEmpty#32] -(42) Exchange -Input [3]: [s_state#9, sum#32, isEmpty#33] -Arguments: hashpartitioning(s_state#9, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(46) CometColumnarExchange +Input [3]: [s_state#9, sum#31, isEmpty#32] +Arguments: hashpartitioning(s_state#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(43) HashAggregate [codegen id : 13] -Input [3]: [s_state#9, sum#32, isEmpty#33] +(47) ColumnarToRow [codegen id : 11] +Input [3]: [s_state#9, sum#31, isEmpty#32] + +(48) HashAggregate [codegen id : 11] +Input [3]: [s_state#9, sum#31, isEmpty#32] 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#9, null AS s_county#36, 0 AS g_state#37, 1 AS g_county#38, 1 AS lochierarchy#39] +Functions [1]: [sum(total_sum#28)] +Aggregate Attributes [1]: [sum(total_sum#28)#33] +Results [6]: [sum(total_sum#28)#33 AS total_sum#34, s_state#9, null AS s_county#35, 0 AS g_state#36, 1 AS g_county#37, 1 AS lochierarchy#38] + +(49) ReusedExchange [Reuses operator id: 38] +Output [3]: [s_state#9, s_county#8, sum#39] -(44) ReusedExchange [Reuses operator id: 37] -Output [3]: [s_state#9, s_county#8, sum#40] +(50) ColumnarToRow [codegen id : 16] +Input [3]: [s_state#9, s_county#8, sum#39] -(45) HashAggregate [codegen id : 19] -Input [3]: [s_state#9, s_county#8, sum#40] +(51) HashAggregate [codegen id : 16] +Input [3]: [s_state#9, s_county#8, sum#39] 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] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#22] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#22,17,2) AS total_sum#28] -(46) HashAggregate [codegen id : 19] -Input [1]: [total_sum#29] +(52) HashAggregate [codegen id : 16] +Input [1]: [total_sum#28] Keys: [] -Functions [1]: [partial_sum(total_sum#29)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [2]: [sum#43, isEmpty#44] +Functions [1]: [partial_sum(total_sum#28)] +Aggregate Attributes [2]: [sum#40, isEmpty#41] +Results [2]: [sum#42, isEmpty#43] + +(53) RowToColumnar +Input [2]: [sum#42, isEmpty#43] + +(54) CometColumnarExchange +Input [2]: [sum#42, isEmpty#43] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(47) Exchange -Input [2]: [sum#43, isEmpty#44] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +(55) ColumnarToRow [codegen id : 17] +Input [2]: [sum#42, isEmpty#43] -(48) HashAggregate [codegen id : 20] -Input [2]: [sum#43, isEmpty#44] +(56) HashAggregate [codegen id : 17] +Input [2]: [sum#42, isEmpty#43] 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] +Functions [1]: [sum(total_sum#28)] +Aggregate Attributes [1]: [sum(total_sum#28)#44] +Results [6]: [sum(total_sum#28)#44 AS total_sum#45, null AS s_state#46, null AS s_county#47, 1 AS g_state#48, 1 AS g_county#49, 2 AS lochierarchy#50] -(49) Union +(57) Union -(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] +(58) HashAggregate [codegen id : 18] +Input [6]: [total_sum#23, s_state#9, s_county#8, g_state#24, g_county#25, lochierarchy#26] +Keys [6]: [total_sum#23, s_state#9, s_county#8, g_state#24, g_county#25, lochierarchy#26] Functions: [] Aggregate Attributes: [] -Results [6]: [total_sum#24, s_state#9, s_county#8, g_state#25, g_county#26, lochierarchy#27] +Results [6]: [total_sum#23, s_state#9, s_county#8, g_state#24, g_county#25, lochierarchy#26] -(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] +(59) RowToColumnar +Input [6]: [total_sum#23, s_state#9, s_county#8, g_state#24, g_county#25, lochierarchy#26] -(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] +(60) CometColumnarExchange +Input [6]: [total_sum#23, s_state#9, s_county#8, g_state#24, g_county#25, lochierarchy#26] +Arguments: hashpartitioning(total_sum#23, s_state#9, s_county#8, g_state#24, g_county#25, lochierarchy#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(61) CometHashAggregate +Input [6]: [total_sum#23, s_state#9, s_county#8, g_state#24, g_county#25, lochierarchy#26] +Keys [6]: [total_sum#23, s_state#9, s_county#8, g_state#24, g_county#25, lochierarchy#26] Functions: [] -Aggregate Attributes: [] -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] -(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] +(62) CometColumnarExchange +Input [5]: [total_sum#23, s_state#9, s_county#8, lochierarchy#26, _w0#51] +Arguments: hashpartitioning(lochierarchy#26, _w0#51, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(63) CometSort +Input [5]: [total_sum#23, s_state#9, s_county#8, lochierarchy#26, _w0#51] +Arguments: [total_sum#23, s_state#9, s_county#8, lochierarchy#26, _w0#51], [lochierarchy#26 ASC NULLS FIRST, _w0#51 ASC NULLS FIRST, total_sum#23 DESC NULLS LAST] -(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 +(64) ColumnarToRow [codegen id : 19] +Input [5]: [total_sum#23, s_state#9, s_county#8, lochierarchy#26, _w0#51] -(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] +(65) Window +Input [5]: [total_sum#23, s_state#9, s_county#8, lochierarchy#26, _w0#51] +Arguments: [rank(total_sum#23) windowspecdefinition(lochierarchy#26, _w0#51, total_sum#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#52], [lochierarchy#26, _w0#51], [total_sum#23 DESC NULLS LAST] -(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] +(66) Project [codegen id : 20] +Output [5]: [total_sum#23, s_state#9, s_county#8, lochierarchy#26, rank_within_parent#52] +Input [6]: [total_sum#23, s_state#9, s_county#8, lochierarchy#26, _w0#51, rank_within_parent#52] -(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] +(67) TakeOrderedAndProject +Input [5]: [total_sum#23, s_state#9, s_county#8, lochierarchy#26, rank_within_parent#52] +Arguments: 100, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0) THEN s_state#9 END ASC NULLS FIRST, rank_within_parent#52 ASC NULLS FIRST], [total_sum#23, s_state#9, s_county#8, lochierarchy#26, rank_within_parent#52] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (62) -+- * ColumnarToRow (61) - +- CometProject (60) - +- CometFilter (59) - +- CometScan parquet spark_catalog.default.date_dim (58) +BroadcastExchange (72) ++- * ColumnarToRow (71) + +- CometProject (70) + +- CometFilter (69) + +- CometScan parquet spark_catalog.default.date_dim (68) -(58) Scan parquet spark_catalog.default.date_dim +(68) 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 -(59) CometFilter +(69) 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)) -(60) CometProject +(70) CometProject Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(61) ColumnarToRow [codegen id : 1] +(71) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(62) BroadcastExchange +(72) BroadcastExchange Input [1]: [d_date_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] 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 feca96b18..16b9573cd 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,69 +1,69 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (24) + WholeStageCodegen (20) Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] InputAdapter Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (23) - Sort [lochierarchy,_w0,total_sum] + WholeStageCodegen (19) + ColumnarToRow InputAdapter - Exchange [lochierarchy,_w0] #1 - 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 (21) + CometSort [total_sum,s_state,s_county,lochierarchy,_w0] + CometColumnarExchange [lochierarchy,_w0] #1 + CometHashAggregate [total_sum,s_state,s_county,lochierarchy,_w0,g_state,g_county] + CometColumnarExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 + RowToColumnar + WholeStageCodegen (18) HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] InputAdapter Union - WholeStageCodegen (6) + WholeStageCodegen (5) 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 (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] - ColumnarToRow - InputAdapter - CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - BroadcastHashJoin [s_state,s_state] - ColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WholeStageCodegen (2) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - InputAdapter - Exchange [s_state] #8 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometColumnarExchange [s_state,s_county] #3 + RowToColumnar + WholeStageCodegen (4) + 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] + ColumnarToRow + InputAdapter + CometProject [ss_store_sk,ss_net_profit] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + BroadcastHashJoin [s_state,s_state] + ColumnarToRow + InputAdapter + CometFilter [s_store_sk,s_county,s_state] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (2) + Project [s_state] + Filter [ranking] + InputAdapter + Window [_w0,s_state] + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometSort [s_state,_w0] + CometHashAggregate [s_state,_w0,sum,sum(UnscaledValue(ss_net_profit))] + CometColumnarExchange [s_state] #8 CometHashAggregate [s_state,sum,ss_net_profit] CometProject [ss_net_profit,s_state] CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] @@ -76,21 +76,27 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [d_date_sk] #5 - WholeStageCodegen (13) + WholeStageCodegen (11) HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - InputAdapter - 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 (20) + ColumnarToRow + InputAdapter + CometColumnarExchange [s_state] #10 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] + ColumnarToRow + InputAdapter + ReusedExchange [s_state,s_county,sum] #3 + WholeStageCodegen (17) HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - InputAdapter - 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 - ReusedExchange [s_state,s_county,sum] #3 + ColumnarToRow + InputAdapter + CometColumnarExchange #11 + RowToColumnar + WholeStageCodegen (16) + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] + ColumnarToRow + InputAdapter + ReusedExchange [s_state,s_county,sum] #3 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 8bff19a72..53e1bb5a6 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,71 +1,72 @@ == Physical Plan == -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) +* ColumnarToRow (68) ++- CometTakeOrderedAndProject (67) + +- CometHashAggregate (66) + +- CometColumnarExchange (65) + +- CometHashAggregate (64) + +- CometProject (63) + +- CometSortMergeJoin (62) + :- CometSort (56) + : +- CometColumnarExchange (55) + : +- RowToColumnar (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) + +- CometSort (61) + +- CometColumnarExchange (60) + +- CometProject (59) + +- CometFilter (58) + +- CometScan parquet spark_catalog.default.catalog_returns (57) (1) Scan parquet spark_catalog.default.catalog_sales @@ -320,101 +321,99 @@ Join condition: None 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#30] -(54) Exchange +(54) RowToColumnar 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=3] -(55) Sort [codegen id : 4] +(55) CometColumnarExchange 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 +Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(56) Scan parquet spark_catalog.default.catalog_returns +(56) CometSort +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, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST] + +(57) 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 -(57) CometFilter +(58) 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)) -(58) CometProject +(59) 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] -(59) ColumnarToRow [codegen id : 5] +(60) CometColumnarExchange Input [2]: [cr_item_sk#31, cr_order_number#32] +Arguments: hashpartitioning(cr_item_sk#31, cr_order_number#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(60) Exchange +(61) CometSort 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] +Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31 ASC NULLS FIRST, cr_order_number#32 ASC NULLS FIRST] -(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 +(62) CometSortMergeJoin +Left output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +Right output [2]: [cr_item_sk#31, cr_order_number#32] +Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#31, cr_order_number#32], LeftOuter -(62) SortMergeJoin [codegen id : 7] -Left keys [2]: [cs_item_sk#4, cs_order_number#6] -Right keys [2]: [cr_item_sk#31, cr_order_number#32] -Join type: LeftOuter -Join condition: None - -(63) Project [codegen id : 7] -Output [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +(63) CometProject 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] +Arguments: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -(64) HashAggregate [codegen id : 7] +(64) CometHashAggregate 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#34] -Results [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#35] -(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] +(65) CometColumnarExchange +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, CometColumnarShuffle, [plan_id=5] -(66) HashAggregate [codegen id : 8] -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#35] +(66) CometHashAggregate +Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] Functions [1]: [count(1)] -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] -(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] +(67) CometTakeOrderedAndProject +Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#37 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#35,promo#36,total_cnt#37]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37], 100, [total_cnt#37 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#35, promo#36, total_cnt#37] + +(68) ColumnarToRow [codegen id : 4] +Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -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 +(69) 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 -(69) CometFilter +(70) 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)) -(70) CometProject +(71) 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] -(71) ColumnarToRow [codegen id : 1] +(72) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -(72) BroadcastExchange +(73) 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=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 6cba2d0e6..1022e1a90 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,94 +1,86 @@ -TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo] - 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 (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 (4) - Sort [cs_item_sk,cs_order_number] - InputAdapter - Exchange [cs_item_sk,cs_order_number] #2 - 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] - 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 [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,d_date_sk,d_week_seq] - 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_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,d_date_sk,d_date,d_week_seq] - 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_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,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_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,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_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_sk,i_item_desc] - 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 [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,w_warehouse_sk,w_warehouse_name] - 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_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,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometFilter [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] - 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 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [i_item_sk,i_item_desc] #6 - CometFilter [i_item_sk,i_item_desc] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange [cd_demo_sk] #7 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange [hd_demo_sk] #8 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange [d_date_sk,d_week_seq] #10 - CometFilter [d_date_sk,d_week_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] +WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] + CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count,count(1)] + CometColumnarExchange [i_item_desc,w_warehouse_name,d_week_seq] #1 + CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] + CometProject [w_warehouse_name,i_item_desc,d_week_seq] + CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] + CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + CometColumnarExchange [cs_item_sk,cs_order_number] #2 + RowToColumnar + 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] + 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 [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,d_date_sk,d_week_seq] + 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_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,d_date_sk,d_date,d_week_seq] + 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_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,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_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,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_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_sk,i_item_desc] + 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 [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,w_warehouse_sk,w_warehouse_name] + 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_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,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometFilter [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] + 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 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange [i_item_sk,i_item_desc] #6 + CometFilter [i_item_sk,i_item_desc] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + CometBroadcastExchange [cd_demo_sk] #7 + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + CometBroadcastExchange [hd_demo_sk] #8 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] + CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + CometBroadcastExchange [d_date_sk,d_week_seq] #10 + CometFilter [d_date_sk,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter - BroadcastExchange #12 - WholeStageCodegen (2) + BroadcastExchange #11 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk] - InputAdapter - WholeStageCodegen (6) - Sort [cr_item_sk,cr_order_number] - InputAdapter - Exchange [cr_item_sk,cr_order_number] #13 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk] + CometSort [cr_item_sk,cr_order_number] + CometColumnarExchange [cr_item_sk,cr_order_number] #13 + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] 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 661c552f2..e92f554c5 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 @@ -1,75 +1,72 @@ == Physical Plan == -TakeOrderedAndProject (71) -+- * Project (70) - +- * BroadcastHashJoin Inner BuildRight (69) - :- * 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) - +- * 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) +* ColumnarToRow (68) ++- CometTakeOrderedAndProject (67) + +- CometProject (66) + +- CometBroadcastHashJoin (65) + :- CometProject (50) + : +- CometBroadcastHashJoin (49) + : :- CometBroadcastHashJoin (33) + : : :- CometFilter (16) + : : : +- CometHashAggregate (15) + : : : +- CometColumnarExchange (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) + : : +- CometBroadcastExchange (32) + : : +- CometHashAggregate (31) + : : +- CometColumnarExchange (30) + : : +- CometHashAggregate (29) + : : +- CometProject (28) + : : +- CometBroadcastHashJoin (27) + : : :- CometProject (23) + : : : +- CometBroadcastHashJoin (22) + : : : :- CometFilter (18) + : : : : +- CometScan parquet spark_catalog.default.customer (17) + : : : +- CometBroadcastExchange (21) + : : : +- CometFilter (20) + : : : +- CometScan parquet spark_catalog.default.store_sales (19) + : : +- CometBroadcastExchange (26) + : : +- CometFilter (25) + : : +- CometScan parquet spark_catalog.default.date_dim (24) + : +- CometBroadcastExchange (48) + : +- CometFilter (47) + : +- CometHashAggregate (46) + : +- CometColumnarExchange (45) + : +- CometHashAggregate (44) + : +- CometProject (43) + : +- CometBroadcastHashJoin (42) + : :- CometProject (40) + : : +- CometBroadcastHashJoin (39) + : : :- CometFilter (35) + : : : +- CometScan parquet spark_catalog.default.customer (34) + : : +- CometBroadcastExchange (38) + : : +- CometFilter (37) + : : +- CometScan parquet spark_catalog.default.web_sales (36) + : +- ReusedExchange (41) + +- CometBroadcastExchange (64) + +- CometHashAggregate (63) + +- CometColumnarExchange (62) + +- CometHashAggregate (61) + +- CometProject (60) + +- CometBroadcastHashJoin (59) + :- CometProject (57) + : +- CometBroadcastHashJoin (56) + : :- CometFilter (52) + : : +- CometScan parquet spark_catalog.default.customer (51) + : +- CometBroadcastExchange (55) + : +- CometFilter (54) + : +- CometScan parquet spark_catalog.default.web_sales (53) + +- ReusedExchange (58) (1) Scan parquet spark_catalog.default.customer @@ -137,337 +134,317 @@ Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_yea 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))] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] - -(16) HashAggregate [codegen id : 8] +(15) CometHashAggregate 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))#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] -(17) Filter [codegen id : 8] -Input [2]: [customer_id#13, year_total#14] -Condition : (isnotnull(year_total#14) AND (year_total#14 > 0.00)) +(16) CometFilter +Input [2]: [customer_id#12, year_total#13] +Condition : (isnotnull(year_total#13) AND (year_total#13 > 0.00)) -(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] +(17) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#14, c_customer_id#15, c_first_name#16, c_last_name#17] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(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)) +(18) CometFilter +Input [4]: [c_customer_sk#14, c_customer_id#15, c_first_name#16, c_last_name#17] +Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_customer_id#15)) -(20) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] +(19) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_customer_sk#18, ss_net_paid#19, ss_sold_date_sk#20] 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#20), dynamicpruningexpression(ss_sold_date_sk#20 IN dynamicpruning#21)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(21) CometFilter -Input [3]: [ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] -Condition : isnotnull(ss_customer_sk#19) +(20) CometFilter +Input [3]: [ss_customer_sk#18, ss_net_paid#19, ss_sold_date_sk#20] +Condition : isnotnull(ss_customer_sk#18) -(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] +(21) CometBroadcastExchange +Input [3]: [ss_customer_sk#18, ss_net_paid#19, ss_sold_date_sk#20] +Arguments: [ss_customer_sk#18, ss_net_paid#19, ss_sold_date_sk#20] -(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 +(22) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#14, c_customer_id#15, c_first_name#16, c_last_name#17] +Right output [3]: [ss_customer_sk#18, ss_net_paid#19, ss_sold_date_sk#20] +Arguments: [c_customer_sk#14], [ss_customer_sk#18], Inner, BuildRight -(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] +(23) CometProject +Input [7]: [c_customer_sk#14, c_customer_id#15, c_first_name#16, c_last_name#17, ss_customer_sk#18, ss_net_paid#19, ss_sold_date_sk#20] +Arguments: [c_customer_id#15, c_first_name#16, c_last_name#17, ss_net_paid#19, ss_sold_date_sk#20], [c_customer_id#15, c_first_name#16, c_last_name#17, ss_net_paid#19, ss_sold_date_sk#20] -(25) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#23, d_year#24] +(24) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#22, d_year#23] 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 -(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 - -(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] +(25) CometFilter +Input [2]: [d_date_sk#22, d_year#23] +Condition : (((isnotnull(d_year#23) AND (d_year#23 = 2002)) AND d_year#23 IN (2001,2002)) AND isnotnull(d_date_sk#22)) + +(26) CometBroadcastExchange +Input [2]: [d_date_sk#22, d_year#23] +Arguments: [d_date_sk#22, d_year#23] + +(27) CometBroadcastHashJoin +Left output [5]: [c_customer_id#15, c_first_name#16, c_last_name#17, ss_net_paid#19, ss_sold_date_sk#20] +Right output [2]: [d_date_sk#22, d_year#23] +Arguments: [ss_sold_date_sk#20], [d_date_sk#22], Inner, BuildRight + +(28) CometProject +Input [7]: [c_customer_id#15, c_first_name#16, c_last_name#17, ss_net_paid#19, ss_sold_date_sk#20, d_date_sk#22, d_year#23] +Arguments: [c_customer_id#15, c_first_name#16, c_last_name#17, ss_net_paid#19, d_year#23], [c_customer_id#15, c_first_name#16, c_last_name#17, ss_net_paid#19, d_year#23] + +(29) CometHashAggregate +Input [5]: [c_customer_id#15, c_first_name#16, c_last_name#17, ss_net_paid#19, d_year#23] +Keys [4]: [c_customer_id#15, c_first_name#16, c_last_name#17, d_year#23] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#19))] + +(30) CometColumnarExchange +Input [5]: [c_customer_id#15, c_first_name#16, c_last_name#17, d_year#23, sum#24] +Arguments: hashpartitioning(c_customer_id#15, c_first_name#16, c_last_name#17, d_year#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(31) CometHashAggregate +Input [5]: [c_customer_id#15, c_first_name#16, c_last_name#17, d_year#23, sum#24] +Keys [4]: [c_customer_id#15, c_first_name#16, c_last_name#17, d_year#23] +Functions [1]: [sum(UnscaledValue(ss_net_paid#19))] + +(32) CometBroadcastExchange +Input [4]: [customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28] +Arguments: [customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28] + +(33) CometBroadcastHashJoin +Left output [2]: [customer_id#12, year_total#13] +Right output [4]: [customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28] +Arguments: [customer_id#12], [customer_id#25], Inner, BuildRight + +(34) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#29, c_customer_id#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_customer_id)] ReadSchema: struct -(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)) +(35) CometFilter +Input [4]: [c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] +Condition : (isnotnull(c_customer_sk#29) AND isnotnull(c_customer_id#30)) -(38) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#34, ws_net_paid#35, ws_sold_date_sk#36] +(36) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#37)] +PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#36)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(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) +(37) CometFilter +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Condition : isnotnull(ws_bill_customer_sk#33) + +(38) CometBroadcastExchange +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Arguments: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] -(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] +(39) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] +Right output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Arguments: [c_customer_sk#29], [ws_bill_customer_sk#33], Inner, BuildRight -(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 +(40) CometProject +Input [7]: [c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32, ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Arguments: [c_customer_id#30, c_first_name#31, c_last_name#32, ws_net_paid#34, ws_sold_date_sk#35], [c_customer_id#30, c_first_name#31, c_last_name#32, ws_net_paid#34, ws_sold_date_sk#35] -(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] +(41) ReusedExchange [Reuses operator id: 10] +Output [2]: [d_date_sk#37, d_year#38] -(43) ReusedExchange [Reuses operator id: 10] -Output [2]: [d_date_sk#38, d_year#39] +(42) CometBroadcastHashJoin +Left output [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, ws_net_paid#34, ws_sold_date_sk#35] +Right output [2]: [d_date_sk#37, d_year#38] +Arguments: [ws_sold_date_sk#35], [d_date_sk#37], Inner, BuildRight -(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 +(43) CometProject +Input [7]: [c_customer_id#30, c_first_name#31, c_last_name#32, ws_net_paid#34, ws_sold_date_sk#35, d_date_sk#37, d_year#38] +Arguments: [c_customer_id#30, c_first_name#31, c_last_name#32, ws_net_paid#34, d_year#38], [c_customer_id#30, c_first_name#31, c_last_name#32, ws_net_paid#34, d_year#38] -(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] +(44) CometHashAggregate +Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, ws_net_paid#34, d_year#38] +Keys [4]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#38] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#34))] + +(45) CometColumnarExchange +Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#38, sum#39] +Arguments: hashpartitioning(c_customer_id#30, c_first_name#31, c_last_name#32, d_year#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (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] - -(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 - -(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] - -(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] +Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#38, sum#39] +Keys [4]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#38] +Functions [1]: [sum(UnscaledValue(ws_net_paid#34))] + +(47) CometFilter +Input [2]: [customer_id#40, year_total#41] +Condition : (isnotnull(year_total#41) AND (year_total#41 > 0.00)) + +(48) CometBroadcastExchange +Input [2]: [customer_id#40, year_total#41] +Arguments: [customer_id#40, year_total#41] + +(49) CometBroadcastHashJoin +Left output [6]: [customer_id#12, year_total#13, customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28] +Right output [2]: [customer_id#40, year_total#41] +Arguments: [customer_id#12], [customer_id#40], Inner, BuildRight + +(50) CometProject +Input [8]: [customer_id#12, year_total#13, customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28, customer_id#40, year_total#41] +Arguments: [customer_id#12, year_total#13, customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28, year_total#41], [customer_id#12, year_total#13, customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28, year_total#41] + +(51) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#42, c_customer_id#43, c_first_name#44, c_last_name#45] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(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)) +(52) CometFilter +Input [4]: [c_customer_sk#42, c_customer_id#43, c_first_name#44, c_last_name#45] +Condition : (isnotnull(c_customer_sk#42) AND isnotnull(c_customer_id#43)) -(56) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] +(53) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_customer_sk#46, ws_net_paid#47, ws_sold_date_sk#48] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#50), dynamicpruningexpression(ws_sold_date_sk#50 IN dynamicpruning#51)] +PartitionFilters: [isnotnull(ws_sold_date_sk#48), dynamicpruningexpression(ws_sold_date_sk#48 IN dynamicpruning#49)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(57) CometFilter -Input [3]: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] -Condition : isnotnull(ws_bill_customer_sk#48) +(54) CometFilter +Input [3]: [ws_bill_customer_sk#46, ws_net_paid#47, ws_sold_date_sk#48] +Condition : isnotnull(ws_bill_customer_sk#46) -(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] +(55) CometBroadcastExchange +Input [3]: [ws_bill_customer_sk#46, ws_net_paid#47, ws_sold_date_sk#48] +Arguments: [ws_bill_customer_sk#46, ws_net_paid#47, ws_sold_date_sk#48] + +(56) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#42, c_customer_id#43, c_first_name#44, c_last_name#45] +Right output [3]: [ws_bill_customer_sk#46, ws_net_paid#47, ws_sold_date_sk#48] +Arguments: [c_customer_sk#42], [ws_bill_customer_sk#46], Inner, BuildRight + +(57) CometProject +Input [7]: [c_customer_sk#42, c_customer_id#43, c_first_name#44, c_last_name#45, ws_bill_customer_sk#46, ws_net_paid#47, ws_sold_date_sk#48] +Arguments: [c_customer_id#43, c_first_name#44, c_last_name#45, ws_net_paid#47, ws_sold_date_sk#48], [c_customer_id#43, c_first_name#44, c_last_name#45, ws_net_paid#47, ws_sold_date_sk#48] + +(58) ReusedExchange [Reuses operator id: 26] +Output [2]: [d_date_sk#50, d_year#51] (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 +Left output [5]: [c_customer_id#43, c_first_name#44, c_last_name#45, ws_net_paid#47, ws_sold_date_sk#48] +Right output [2]: [d_date_sk#50, d_year#51] +Arguments: [ws_sold_date_sk#48], [d_date_sk#50], Inner, BuildRight (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) ReusedExchange [Reuses operator id: 27] -Output [2]: [d_date_sk#52, d_year#53] - -(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) 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) 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) 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#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 : 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#55, year_total#56] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] - -(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#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 : 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#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] +Input [7]: [c_customer_id#43, c_first_name#44, c_last_name#45, ws_net_paid#47, ws_sold_date_sk#48, d_date_sk#50, d_year#51] +Arguments: [c_customer_id#43, c_first_name#44, c_last_name#45, ws_net_paid#47, d_year#51], [c_customer_id#43, c_first_name#44, c_last_name#45, ws_net_paid#47, d_year#51] + +(61) CometHashAggregate +Input [5]: [c_customer_id#43, c_first_name#44, c_last_name#45, ws_net_paid#47, d_year#51] +Keys [4]: [c_customer_id#43, c_first_name#44, c_last_name#45, d_year#51] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#47))] + +(62) CometColumnarExchange +Input [5]: [c_customer_id#43, c_first_name#44, c_last_name#45, d_year#51, sum#52] +Arguments: hashpartitioning(c_customer_id#43, c_first_name#44, c_last_name#45, d_year#51, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(63) CometHashAggregate +Input [5]: [c_customer_id#43, c_first_name#44, c_last_name#45, d_year#51, sum#52] +Keys [4]: [c_customer_id#43, c_first_name#44, c_last_name#45, d_year#51] +Functions [1]: [sum(UnscaledValue(ws_net_paid#47))] + +(64) CometBroadcastExchange +Input [2]: [customer_id#53, year_total#54] +Arguments: [customer_id#53, year_total#54] + +(65) CometBroadcastHashJoin +Left output [7]: [customer_id#12, year_total#13, customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28, year_total#41] +Right output [2]: [customer_id#53, year_total#54] +Arguments: [customer_id#12], [customer_id#53], Inner, (CASE WHEN (year_total#41 > 0.00) THEN (year_total#54 / year_total#41) END > CASE WHEN (year_total#13 > 0.00) THEN (year_total#28 / year_total#13) END), BuildRight + +(66) CometProject +Input [9]: [customer_id#12, year_total#13, customer_id#25, customer_first_name#26, customer_last_name#27, year_total#28, year_total#41, customer_id#53, year_total#54] +Arguments: [customer_id#25, customer_first_name#26, customer_last_name#27], [customer_id#25, customer_first_name#26, customer_last_name#27] + +(67) CometTakeOrderedAndProject +Input [3]: [customer_id#25, customer_first_name#26, customer_last_name#27] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_first_name#26 ASC NULLS FIRST,customer_id#25 ASC NULLS FIRST,customer_last_name#27 ASC NULLS FIRST], output=[customer_id#25,customer_first_name#26,customer_last_name#27]), [customer_id#25, customer_first_name#26, customer_last_name#27], 100, [customer_first_name#26 ASC NULLS FIRST, customer_id#25 ASC NULLS FIRST, customer_last_name#27 ASC NULLS FIRST], [customer_id#25, customer_first_name#26, customer_last_name#27] + +(68) ColumnarToRow [codegen id : 1] +Input [3]: [customer_id#25, customer_first_name#26, customer_last_name#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (75) -+- * ColumnarToRow (74) - +- CometFilter (73) - +- CometScan parquet spark_catalog.default.date_dim (72) +BroadcastExchange (72) ++- * ColumnarToRow (71) + +- CometFilter (70) + +- CometScan parquet spark_catalog.default.date_dim (69) -(72) Scan parquet spark_catalog.default.date_dim +(69) 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 -(73) CometFilter +(70) 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)) -(74) ColumnarToRow [codegen id : 1] +(71) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_year#10] -(75) BroadcastExchange +(72) BroadcastExchange Input [2]: [d_date_sk#9, d_year#10] -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=5] -Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#21 IN dynamicpruning#22 -BroadcastExchange (79) -+- * ColumnarToRow (78) - +- CometFilter (77) - +- CometScan parquet spark_catalog.default.date_dim (76) +Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#21 +BroadcastExchange (76) ++- * ColumnarToRow (75) + +- CometFilter (74) + +- CometScan parquet spark_catalog.default.date_dim (73) -(76) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#23, d_year#24] +(73) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#22, d_year#23] 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#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)) +(74) CometFilter +Input [2]: [d_date_sk#22, d_year#23] +Condition : (((isnotnull(d_year#23) AND (d_year#23 = 2002)) AND d_year#23 IN (2001,2002)) AND isnotnull(d_date_sk#22)) -(78) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#23, d_year#24] +(75) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#22, d_year#23] -(79) BroadcastExchange -Input [2]: [d_date_sk#23, d_year#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] +(76) BroadcastExchange +Input [2]: [d_date_sk#22, d_year#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -Subquery:3 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#8 -Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#50 IN dynamicpruning#22 +Subquery:4 Hosting operator id = 53 Hosting Expression = ws_sold_date_sk#48 IN dynamicpruning#21 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 e5cf23f50..0b288df94 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,106 +1,86 @@ -TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] - 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] - BroadcastHashJoin [customer_id,customer_id] - BroadcastHashJoin [customer_id,customer_id] - Filter [year_total] - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_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_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - 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,sum,ss_net_paid] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - 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] #11 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 - CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_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 #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] #14 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 - CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_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 +WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] + CometProject [customer_id,customer_first_name,customer_last_name] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total,customer_id,year_total] + CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,customer_id,year_total] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total] + CometFilter [customer_id,year_total] + CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,d_year,sum,sum(UnscaledValue(ss_net_paid))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_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_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,year_total] #5 + CometHashAggregate [customer_id,customer_first_name,customer_last_name,year_total,c_customer_id,c_first_name,c_last_name,d_year,sum,sum(UnscaledValue(ss_net_paid))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #6 + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [customer_id,year_total] #10 + CometFilter [customer_id,year_total] + CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,d_year,sum,sum(UnscaledValue(ws_net_paid))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #11 + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_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 + CometBroadcastExchange [customer_id,year_total] #13 + CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,d_year,sum,sum(UnscaledValue(ws_net_paid))] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #14 + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_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 564e40928..2aeff8de1 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,136 +1,129 @@ == Physical Plan == -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) +TakeOrderedAndProject (125) ++- * Project (124) + +- * SortMergeJoin Inner (123) + :- * ColumnarToRow (67) + : +- CometSort (66) + : +- CometColumnarExchange (65) + : +- CometFilter (64) + : +- CometHashAggregate (63) + : +- CometColumnarExchange (62) + : +- CometHashAggregate (61) + : +- CometHashAggregate (60) + : +- CometColumnarExchange (59) + : +- CometHashAggregate (58) + : +- CometUnion (57) + : :- CometProject (22) + : : +- CometSortMergeJoin (21) + : : :- CometSort (15) + : : : +- CometColumnarExchange (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) + : : +- CometSort (20) + : : +- CometColumnarExchange (19) + : : +- CometProject (18) + : : +- CometFilter (17) + : : +- CometScan parquet spark_catalog.default.catalog_returns (16) + : :- CometProject (39) + : : +- CometSortMergeJoin (38) + : : :- CometSort (32) + : : : +- CometColumnarExchange (31) + : : : +- CometProject (30) + : : : +- CometBroadcastHashJoin (29) + : : : :- CometProject (27) + : : : : +- CometBroadcastHashJoin (26) + : : : : :- CometFilter (24) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (23) + : : : : +- ReusedExchange (25) + : : : +- ReusedExchange (28) + : : +- CometSort (37) + : : +- CometColumnarExchange (36) + : : +- CometProject (35) + : : +- CometFilter (34) + : : +- CometScan parquet spark_catalog.default.store_returns (33) + : +- CometProject (56) + : +- CometSortMergeJoin (55) + : :- CometSort (49) + : : +- CometColumnarExchange (48) + : : +- CometProject (47) + : : +- CometBroadcastHashJoin (46) + : : :- CometProject (44) + : : : +- CometBroadcastHashJoin (43) + : : : :- CometFilter (41) + : : : : +- CometScan parquet spark_catalog.default.web_sales (40) + : : : +- ReusedExchange (42) + : : +- ReusedExchange (45) + : +- CometSort (54) + : +- CometColumnarExchange (53) + : +- CometProject (52) + : +- CometFilter (51) + : +- CometScan parquet spark_catalog.default.web_returns (50) + +- * ColumnarToRow (122) + +- CometSort (121) + +- CometColumnarExchange (120) + +- CometFilter (119) + +- CometHashAggregate (118) + +- CometColumnarExchange (117) + +- CometHashAggregate (116) + +- CometHashAggregate (115) + +- CometColumnarExchange (114) + +- CometHashAggregate (113) + +- CometUnion (112) + :- CometProject (83) + : +- CometSortMergeJoin (82) + : :- CometSort (79) + : : +- CometColumnarExchange (78) + : : +- CometProject (77) + : : +- CometBroadcastHashJoin (76) + : : :- CometProject (72) + : : : +- CometBroadcastHashJoin (71) + : : : :- CometFilter (69) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (68) + : : : +- ReusedExchange (70) + : : +- CometBroadcastExchange (75) + : : +- CometFilter (74) + : : +- CometScan parquet spark_catalog.default.date_dim (73) + : +- CometSort (81) + : +- ReusedExchange (80) + :- CometProject (97) + : +- CometSortMergeJoin (96) + : :- CometSort (93) + : : +- CometColumnarExchange (92) + : : +- CometProject (91) + : : +- CometBroadcastHashJoin (90) + : : :- CometProject (88) + : : : +- CometBroadcastHashJoin (87) + : : : :- CometFilter (85) + : : : : +- CometScan parquet spark_catalog.default.store_sales (84) + : : : +- ReusedExchange (86) + : : +- ReusedExchange (89) + : +- CometSort (95) + : +- ReusedExchange (94) + +- CometProject (111) + +- CometSortMergeJoin (110) + :- CometSort (107) + : +- CometColumnarExchange (106) + : +- CometProject (105) + : +- CometBroadcastHashJoin (104) + : :- CometProject (102) + : : +- CometBroadcastHashJoin (101) + : : :- CometFilter (99) + : : : +- CometScan parquet spark_catalog.default.web_sales (98) + : : +- ReusedExchange (100) + : +- ReusedExchange (103) + +- CometSort (109) + +- ReusedExchange (108) (1) Scan parquet spark_catalog.default.catalog_sales @@ -197,54 +190,47 @@ Arguments: [cs_sold_date_sk#5], [d_date_sk#13], Inner, BuildRight 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] +(14) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(15) Exchange +(15) CometSort 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=1] +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_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST] -(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 - -(17) Scan parquet spark_catalog.default.catalog_returns +(16) 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 -(18) CometFilter +(17) 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)) -(19) CometProject +(18) 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] -(20) ColumnarToRow [codegen id : 3] -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] - -(21) Exchange +(19) CometColumnarExchange 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=2] +Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(22) Sort [codegen id : 4] +(20) CometSort 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 +Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST] -(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 +(21) CometSortMergeJoin +Left 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] +Right output [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] +Arguments: [cs_order_number#2, cs_item_sk#1], [cr_order_number#16, cr_item_sk#15], LeftOuter -(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] +(22) CometProject 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] +Arguments: [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], [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] -(25) Scan parquet spark_catalog.default.store_sales +(23) 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 [] @@ -252,82 +238,75 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(26) CometFilter +(24) 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) -(27) ReusedExchange [Reuses operator id: 6] +(25) 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) CometBroadcastHashJoin +(26) 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) CometProject +(27) 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: 11] +(28) ReusedExchange [Reuses operator id: 11] Output [2]: [d_date_sk#33, d_year#34] -(31) CometBroadcastHashJoin +(29) 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) CometProject +(30) 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] +(31) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=3] -(34) Exchange +(32) CometSort 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=3] +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_ticket_number#23 ASC NULLS FIRST, ss_item_sk#22 ASC NULLS FIRST] -(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 - -(36) Scan parquet spark_catalog.default.store_returns +(33) 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 -(37) CometFilter +(34) 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)) -(38) CometProject +(35) 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] -(39) ColumnarToRow [codegen id : 8] +(36) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=4] -(40) Exchange +(37) CometSort 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=4] +Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST] -(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 +(38) CometSortMergeJoin +Left 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] +Right output [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] +Arguments: [ss_ticket_number#23, ss_item_sk#22], [sr_ticket_number#36, sr_item_sk#35], LeftOuter -(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 - -(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] +(39) CometProject 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] +Arguments: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41], [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] -(44) Scan parquet spark_catalog.default.web_sales +(40) 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 [] @@ -335,457 +314,434 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(45) CometFilter +(41) 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) -(46) ReusedExchange [Reuses operator id: 6] +(42) 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) CometBroadcastHashJoin +(43) 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) CometProject +(44) 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: 11] +(45) ReusedExchange [Reuses operator id: 11] Output [2]: [d_date_sk#53, d_year#54] -(50) CometBroadcastHashJoin +(46) 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) CometProject +(47) 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) 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] - -(53) Exchange +(48) CometColumnarExchange 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] +Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(54) Sort [codegen id : 12] +(49) CometSort 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 +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_order_number#43 ASC NULLS FIRST, ws_item_sk#42 ASC NULLS FIRST] -(55) Scan parquet spark_catalog.default.web_returns +(50) 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 -(56) CometFilter +(51) 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)) -(57) CometProject +(52) 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] -(58) ColumnarToRow [codegen id : 13] +(53) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=6] -(59) Exchange +(54) CometSort 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=6] +Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST] -(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 - -(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 +(55) CometSortMergeJoin +Left 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] +Right output [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] +Arguments: [ws_order_number#43, ws_item_sk#42], [wr_order_number#56, wr_item_sk#55], LeftOuter -(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] +(56) CometProject 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] +Arguments: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61], [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] -(63) Union +(57) CometUnion +Child 0 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] +Child 1 Input [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41] +Child 2 Input [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61] -(64) HashAggregate [codegen id : 16] +(58) CometHashAggregate 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] -(65) Exchange +(59) CometColumnarExchange 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=7] +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, CometColumnarShuffle, [plan_id=7] -(66) HashAggregate [codegen id : 17] +(60) CometHashAggregate 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] -(67) HashAggregate [codegen id : 17] +(61) CometHashAggregate 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] -(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=8] +(62) CometColumnarExchange +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] +Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(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] +(63) CometHashAggregate +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] 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] -(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) +(64) CometFilter +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] +Condition : isnotnull(sales_cnt#64) + +(65) CometColumnarExchange +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(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=9] +(66) CometSort +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] +Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65], [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] -(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 +(67) ColumnarToRow [codegen id : 1] +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -(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] +(68) Scan parquet spark_catalog.default.catalog_sales +Output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#74), dynamicpruningexpression(cs_sold_date_sk#74 IN dynamicpruning#75)] +PartitionFilters: [isnotnull(cs_sold_date_sk#70), dynamicpruningexpression(cs_sold_date_sk#70 IN dynamicpruning#71)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(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) +(69) CometFilter +Input [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] +Condition : isnotnull(cs_item_sk#66) -(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] +(70) ReusedExchange [Reuses operator id: 6] +Output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -(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 +(71) CometBroadcastHashJoin +Left output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] +Right output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] +Arguments: [cs_item_sk#66], [i_item_sk#72], Inner, BuildRight -(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] +(72) CometProject +Input [10]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] +Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -(78) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#81, d_year#82] +(73) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#77, d_year#78] 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) 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) CometBroadcastExchange -Input [2]: [d_date_sk#81, d_year#82] -Arguments: [d_date_sk#81, d_year#82] +(74) CometFilter +Input [2]: [d_date_sk#77, d_year#78] +Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) -(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 +(75) CometBroadcastExchange +Input [2]: [d_date_sk#77, d_year#78] +Arguments: [d_date_sk#77, d_year#78] -(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] +(76) CometBroadcastHashJoin +Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] +Right output [2]: [d_date_sk#77, d_year#78] +Arguments: [cs_sold_date_sk#70], [d_date_sk#77], Inner, BuildRight -(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] +(77) CometProject +Input [11]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_date_sk#77, d_year#78] +Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -(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=10] +(78) CometColumnarExchange +Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] +Arguments: hashpartitioning(cs_order_number#67, cs_item_sk#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(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 +(79) CometSort +Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] +Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_order_number#67 ASC NULLS FIRST, cs_item_sk#66 ASC NULLS FIRST] -(86) ReusedExchange [Reuses operator id: 21] -Output [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] +(80) ReusedExchange [Reuses operator id: 19] +Output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -(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 +(81) CometSort +Input [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] +Arguments: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82], [cr_order_number#80 ASC NULLS FIRST, cr_item_sk#79 ASC NULLS FIRST] -(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 +(82) CometSortMergeJoin +Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] +Right output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] +Arguments: [cs_order_number#67, cs_item_sk#66], [cr_order_number#80, cr_item_sk#79], LeftOuter -(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] +(83) CometProject +Input [13]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78, cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] +Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21], [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, (cs_quantity#68 - coalesce(cr_return_quantity#81, 0)) AS sales_cnt#20, (cs_ext_sales_price#69 - coalesce(cr_return_amount#82, 0.00)) AS sales_amt#21] -(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] +(84) Scan parquet spark_catalog.default.store_sales +Output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#91), dynamicpruningexpression(ss_sold_date_sk#91 IN dynamicpruning#92)] +PartitionFilters: [isnotnull(ss_sold_date_sk#87), dynamicpruningexpression(ss_sold_date_sk#87 IN dynamicpruning#88)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(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) - -(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] +(85) CometFilter +Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] +Condition : isnotnull(ss_item_sk#83) -(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 +(86) ReusedExchange [Reuses operator id: 6] +Output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -(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] +(87) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] +Right output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] +Arguments: [ss_item_sk#83], [i_item_sk#89], Inner, BuildRight -(95) ReusedExchange [Reuses operator id: 80] -Output [2]: [d_date_sk#98, d_year#99] +(88) CometProject +Input [10]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] +Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93], [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -(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 +(89) ReusedExchange [Reuses operator id: 75] +Output [2]: [d_date_sk#94, d_year#95] -(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] +(90) CometBroadcastHashJoin +Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] +Right output [2]: [d_date_sk#94, d_year#95] +Arguments: [ss_sold_date_sk#87], [d_date_sk#94], Inner, BuildRight -(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] +(91) CometProject +Input [11]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_date_sk#94, d_year#95] +Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -(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] +(92) CometColumnarExchange +Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] +Arguments: hashpartitioning(ss_ticket_number#84, ss_item_sk#83, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [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 +(93) CometSort +Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] +Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_ticket_number#84 ASC NULLS FIRST, ss_item_sk#83 ASC NULLS FIRST] -(101) ReusedExchange [Reuses operator id: 40] -Output [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] +(94) ReusedExchange [Reuses operator id: 36] +Output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -(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 +(95) CometSort +Input [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] +Arguments: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99], [sr_ticket_number#97 ASC NULLS FIRST, sr_item_sk#96 ASC NULLS FIRST] -(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 +(96) CometSortMergeJoin +Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] +Right output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] +Arguments: [ss_ticket_number#84, ss_item_sk#83], [sr_ticket_number#97, sr_item_sk#96], LeftOuter -(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] +(97) CometProject +Input [13]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95, sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] +Arguments: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41], [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, (ss_quantity#85 - coalesce(sr_return_quantity#98, 0)) AS sales_cnt#40, (ss_ext_sales_price#86 - coalesce(sr_return_amt#99, 0.00)) AS sales_amt#41] -(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] +(98) Scan parquet spark_catalog.default.web_sales +Output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#108), dynamicpruningexpression(ws_sold_date_sk#108 IN dynamicpruning#109)] +PartitionFilters: [isnotnull(ws_sold_date_sk#104), dynamicpruningexpression(ws_sold_date_sk#104 IN dynamicpruning#105)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(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) - -(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] +(99) CometFilter +Input [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] +Condition : isnotnull(ws_item_sk#100) -(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 +(100) ReusedExchange [Reuses operator id: 6] +Output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -(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] +(101) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] +Right output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] +Arguments: [ws_item_sk#100], [i_item_sk#106], Inner, BuildRight -(110) ReusedExchange [Reuses operator id: 80] -Output [2]: [d_date_sk#115, d_year#116] +(102) CometProject +Input [10]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] +Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110], [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -(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 +(103) ReusedExchange [Reuses operator id: 75] +Output [2]: [d_date_sk#111, d_year#112] -(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] +(104) CometBroadcastHashJoin +Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] +Right output [2]: [d_date_sk#111, d_year#112] +Arguments: [ws_sold_date_sk#104], [d_date_sk#111], Inner, BuildRight -(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] +(105) CometProject +Input [11]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_date_sk#111, d_year#112] +Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -(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=12] +(106) CometColumnarExchange +Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] +Arguments: hashpartitioning(ws_order_number#101, ws_item_sk#100, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(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 +(107) CometSort +Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] +Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_order_number#101 ASC NULLS FIRST, ws_item_sk#100 ASC NULLS FIRST] -(116) ReusedExchange [Reuses operator id: 59] -Output [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] +(108) ReusedExchange [Reuses operator id: 53] +Output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -(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 +(109) CometSort +Input [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] +Arguments: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116], [wr_order_number#114 ASC NULLS FIRST, wr_item_sk#113 ASC NULLS FIRST] -(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 +(110) CometSortMergeJoin +Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] +Right output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] +Arguments: [ws_order_number#101, ws_item_sk#100], [wr_order_number#114, wr_item_sk#113], LeftOuter -(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] +(111) CometProject +Input [13]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112, wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] +Arguments: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61], [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, (ws_quantity#102 - coalesce(wr_return_quantity#115, 0)) AS sales_cnt#60, (ws_ext_sales_price#103 - coalesce(wr_return_amt#116, 0.00)) AS sales_amt#61] -(120) Union +(112) CometUnion +Child 0 Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] +Child 1 Input [7]: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41] +Child 2 Input [7]: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61] -(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] +(113) CometHashAggregate +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] +Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 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] -(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=13] +(114) CometColumnarExchange +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] +Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(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] +(115) CometHashAggregate +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] +Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 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] -(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] +(116) CometHashAggregate +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] +Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] 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] -(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=14] +(117) CometColumnarExchange +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] +Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] -(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] +(118) CometHashAggregate +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] +Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] 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] -(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) +(119) CometFilter +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] +Condition : isnotnull(sales_cnt#118) + +(120) CometColumnarExchange +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] +Arguments: hashpartitioning(i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] -(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=15] +(121) CometSort +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] +Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119], [i_brand_id#73 ASC NULLS FIRST, i_class_id#74 ASC NULLS FIRST, i_category_id#75 ASC NULLS FIRST, i_manufact_id#76 ASC NULLS FIRST] -(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 +(122) ColumnarToRow [codegen id : 2] +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -(130) SortMergeJoin [codegen id : 39] +(123) SortMergeJoin [codegen id : 3] 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] +Right keys [4]: [i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] Join type: Inner -Join condition: ((cast(sales_cnt#68 as decimal(17,2)) / cast(sales_cnt#123 as decimal(17,2))) < 0.90000000000000000000) +Join condition: ((cast(sales_cnt#64 as decimal(17,2)) / cast(sales_cnt#118 as decimal(17,2))) < 0.90000000000000000000) -(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] +(124) Project [codegen id : 3] +Output [10]: [d_year#78 AS prev_year#120, d_year#14 AS year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#118 AS prev_yr_cnt#122, sales_cnt#64 AS curr_yr_cnt#123, (sales_cnt#64 - sales_cnt#118) AS sales_cnt_diff#124, (sales_amt#65 - sales_amt#119) AS sales_amt_diff#125] +Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65, d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -(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] +(125) TakeOrderedAndProject +Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] +Arguments: 100, [sales_cnt_diff#124 ASC NULLS FIRST, sales_amt_diff#125 ASC NULLS FIRST], [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (136) -+- * ColumnarToRow (135) - +- CometFilter (134) - +- CometScan parquet spark_catalog.default.date_dim (133) +BroadcastExchange (129) ++- * ColumnarToRow (128) + +- CometFilter (127) + +- CometScan parquet spark_catalog.default.date_dim (126) -(133) Scan parquet spark_catalog.default.date_dim +(126) 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 -(134) CometFilter +(127) 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)) -(135) ColumnarToRow [codegen id : 1] +(128) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] -(136) BroadcastExchange +(129) BroadcastExchange Input [2]: [d_date_sk#13, d_year#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 -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) +Subquery:4 Hosting operator id = 68 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 +BroadcastExchange (133) ++- * ColumnarToRow (132) + +- CometFilter (131) + +- CometScan parquet spark_catalog.default.date_dim (130) -(137) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#81, d_year#82] +(130) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#77, d_year#78] 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 -(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)) +(131) CometFilter +Input [2]: [d_date_sk#77, d_year#78] +Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) -(139) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#81, d_year#82] +(132) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#77, d_year#78] -(140) BroadcastExchange -Input [2]: [d_date_sk#81, d_year#82] +(133) BroadcastExchange +Input [2]: [d_date_sk#77, d_year#78] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] -Subquery:5 Hosting operator id = 90 Hosting Expression = ss_sold_date_sk#91 IN dynamicpruning#75 +Subquery:5 Hosting operator id = 84 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#71 -Subquery:6 Hosting operator id = 105 Hosting Expression = ws_sold_date_sk#108 IN dynamicpruning#75 +Subquery:6 Hosting operator id = 98 Hosting Expression = ws_sold_date_sk#104 IN dynamicpruning#71 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 2aecc9e98..9b187bca1 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,229 +1,150 @@ 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 (39) + WholeStageCodegen (3) 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 (19) - Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - 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 (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 (16) - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - InputAdapter - Union - 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 (2) - Sort [cs_order_number,cs_item_sk] - InputAdapter - Exchange [cs_order_number,cs_item_sk] #4 - 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_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,d_date_sk,d_year] - 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,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_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 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,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 [d_date_sk,d_year] #7 - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - WholeStageCodegen (4) - Sort [cr_order_number,cr_item_sk] - InputAdapter - 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_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_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 (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 (7) - Sort [ss_ticket_number,ss_item_sk] - 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_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,d_date_sk,d_year] - 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,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_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 - 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 (9) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - 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_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_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 (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 (12) - Sort [ws_order_number,ws_item_sk] - 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_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,d_date_sk,d_year] - 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,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_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 - 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 (14) - Sort [wr_order_number,wr_item_sk] - InputAdapter - 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_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 + CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum,sum,sum(sales_cnt),sum(UnscaledValue(sales_amt))] + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum,sales_cnt,sales_amt] + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometSort [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] + CometColumnarExchange [cs_order_number,cs_item_sk] #4 + 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_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,d_date_sk,d_year] + 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,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_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 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,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 [d_date_sk,d_year] #7 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometColumnarExchange [cr_order_number,cr_item_sk] #8 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometSort [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] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #9 + 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_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,d_date_sk,d_year] + 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,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_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 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #7 + CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometColumnarExchange [sr_ticket_number,sr_item_sk] #10 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometSort [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] + CometColumnarExchange [ws_order_number,ws_item_sk] #11 + 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_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,d_date_sk,d_year] + 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,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_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 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #7 + CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometColumnarExchange [wr_order_number,wr_item_sk] #12 + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_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 (38) - Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] + WholeStageCodegen (2) + ColumnarToRow InputAdapter - 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] #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] #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 (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 (21) - Sort [cs_order_number,cs_item_sk] - 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_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,d_date_sk,d_year] - 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,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_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 #17 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometBroadcastExchange [d_date_sk,d_year] #18 - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - WholeStageCodegen (23) - Sort [cr_order_number,cr_item_sk] - InputAdapter - 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 (26) - Sort [ss_ticket_number,ss_item_sk] - 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_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,d_date_sk,d_year] - 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,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_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 - 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 (28) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - 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 (31) - Sort [ws_order_number,ws_item_sk] - 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_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,d_date_sk,d_year] - 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,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_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 - 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 (33) - Sort [wr_order_number,wr_item_sk] - InputAdapter - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #12 + CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 + CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum,sum,sum(sales_cnt),sum(UnscaledValue(sales_amt))] + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum,sales_cnt,sales_amt] + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometSort [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] + CometColumnarExchange [cs_order_number,cs_item_sk] #16 + 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_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,d_date_sk,d_year] + 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,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_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 #17 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + CometBroadcastExchange [d_date_sk,d_year] #18 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #8 + CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometSort [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] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #19 + 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_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,d_date_sk,d_year] + 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,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_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 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #18 + CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #10 + CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometSort [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] + CometColumnarExchange [ws_order_number,ws_item_sk] #20 + 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_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,d_date_sk,d_year] + 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,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_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 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #18 + CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + 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 168e394dd..ec37a5df5 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,103 +1,111 @@ == Physical Plan == -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) +* ColumnarToRow (107) ++- CometTakeOrderedAndProject (106) + +- CometHashAggregate (105) + +- CometColumnarExchange (104) + +- RowToColumnar (103) + +- * HashAggregate (102) + +- Union (101) + :- * HashAggregate (84) + : +- * ColumnarToRow (83) + : +- CometColumnarExchange (82) + : +- RowToColumnar (81) + : +- * HashAggregate (80) + : +- Union (79) + : :- * ColumnarToRow (31) + : : +- CometProject (30) + : : +- CometBroadcastHashJoin (29) + : : :- CometHashAggregate (16) + : : : +- CometColumnarExchange (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) + : : +- CometBroadcastExchange (28) + : : +- CometHashAggregate (27) + : : +- CometColumnarExchange (26) + : : +- CometHashAggregate (25) + : : +- CometProject (24) + : : +- CometBroadcastHashJoin (23) + : : :- CometProject (21) + : : : +- CometBroadcastHashJoin (20) + : : : :- CometFilter (18) + : : : : +- CometScan parquet spark_catalog.default.store_returns (17) + : : : +- ReusedExchange (19) + : : +- ReusedExchange (22) + : :- * Project (50) + : : +- * BroadcastNestedLoopJoin Inner BuildLeft (49) + : : :- BroadcastExchange (40) + : : : +- * ColumnarToRow (39) + : : : +- CometHashAggregate (38) + : : : +- CometColumnarExchange (37) + : : : +- CometHashAggregate (36) + : : : +- CometProject (35) + : : : +- CometBroadcastHashJoin (34) + : : : :- CometScan parquet spark_catalog.default.catalog_sales (32) + : : : +- ReusedExchange (33) + : : +- * ColumnarToRow (48) + : : +- CometHashAggregate (47) + : : +- CometColumnarExchange (46) + : : +- CometHashAggregate (45) + : : +- CometProject (44) + : : +- CometBroadcastHashJoin (43) + : : :- CometScan parquet spark_catalog.default.catalog_returns (41) + : : +- ReusedExchange (42) + : +- * ColumnarToRow (78) + : +- CometProject (77) + : +- CometBroadcastHashJoin (76) + : :- CometHashAggregate (63) + : : +- CometColumnarExchange (62) + : : +- CometHashAggregate (61) + : : +- CometProject (60) + : : +- CometBroadcastHashJoin (59) + : : :- CometProject (55) + : : : +- CometBroadcastHashJoin (54) + : : : :- CometFilter (52) + : : : : +- CometScan parquet spark_catalog.default.web_sales (51) + : : : +- ReusedExchange (53) + : : +- CometBroadcastExchange (58) + : : +- CometFilter (57) + : : +- CometScan parquet spark_catalog.default.web_page (56) + : +- CometBroadcastExchange (75) + : +- CometHashAggregate (74) + : +- CometColumnarExchange (73) + : +- CometHashAggregate (72) + : +- CometProject (71) + : +- CometBroadcastHashJoin (70) + : :- CometProject (68) + : : +- CometBroadcastHashJoin (67) + : : :- CometFilter (65) + : : : +- CometScan parquet spark_catalog.default.web_returns (64) + : : +- ReusedExchange (66) + : +- ReusedExchange (69) + :- * HashAggregate (92) + : +- * ColumnarToRow (91) + : +- CometColumnarExchange (90) + : +- RowToColumnar (89) + : +- * HashAggregate (88) + : +- * HashAggregate (87) + : +- * ColumnarToRow (86) + : +- ReusedExchange (85) + +- * HashAggregate (100) + +- * ColumnarToRow (99) + +- CometColumnarExchange (98) + +- RowToColumnar (97) + +- * HashAggregate (96) + +- * HashAggregate (95) + +- * ColumnarToRow (94) + +- ReusedExchange (93) (1) Scan parquet spark_catalog.default.store_sales @@ -169,453 +177,461 @@ 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))] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarExchange Input [3]: [s_store_sk#8, sum#9, sum#10] +Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(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] +(16) CometHashAggregate 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))#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] -(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] +(17) Scan parquet spark_catalog.default.store_returns +Output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#18), dynamicpruningexpression(sr_returned_date_sk#18 IN dynamicpruning#19)] +PartitionFilters: [isnotnull(sr_returned_date_sk#14), dynamicpruningexpression(sr_returned_date_sk#14 IN dynamicpruning#15)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(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) - -(20) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#20] - -(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 - -(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] - -(23) ReusedExchange [Reuses operator id: 11] -Output [1]: [s_store_sk#21] - -(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 - -(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] - -(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 +(18) CometFilter +Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] +Condition : isnotnull(sr_store_sk#11) + +(19) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#16] + +(20) CometBroadcastHashJoin +Left output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] +Right output [1]: [d_date_sk#16] +Arguments: [sr_returned_date_sk#14], [d_date_sk#16], Inner, BuildRight + +(21) CometProject +Input [5]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14, d_date_sk#16] +Arguments: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13], [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] + +(22) ReusedExchange [Reuses operator id: 11] +Output [1]: [s_store_sk#17] + +(23) CometBroadcastHashJoin +Left output [3]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] +Right output [1]: [s_store_sk#17] +Arguments: [sr_store_sk#11], [s_store_sk#17], Inner, BuildRight + +(24) CometProject +Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, s_store_sk#17] +Arguments: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17], [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] + +(25) CometHashAggregate +Input [3]: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] +Keys [1]: [s_store_sk#17] +Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#12)), partial_sum(UnscaledValue(sr_net_loss#13))] + +(26) CometColumnarExchange +Input [3]: [s_store_sk#17, sum#18, sum#19] +Arguments: hashpartitioning(s_store_sk#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(27) CometHashAggregate +Input [3]: [s_store_sk#17, sum#18, sum#19] +Keys [1]: [s_store_sk#17] +Functions [2]: [sum(UnscaledValue(sr_return_amt#12)), sum(UnscaledValue(sr_net_loss#13))] + +(28) CometBroadcastExchange +Input [3]: [s_store_sk#17, returns#20, profit_loss#21] +Arguments: [s_store_sk#17, returns#20, profit_loss#21] + +(29) CometBroadcastHashJoin +Left output [3]: [s_store_sk#8, sales#22, profit#23] +Right output [3]: [s_store_sk#17, returns#20, profit_loss#21] +Arguments: [s_store_sk#8], [s_store_sk#17], LeftOuter, BuildRight -(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] +(30) CometProject +Input [6]: [s_store_sk#8, sales#22, profit#23, s_store_sk#17, returns#20, profit_loss#21] +Arguments: [channel#24, id#25, sales#22, returns#26, profit#27], [store channel AS channel#24, s_store_sk#8 AS id#25, sales#22, coalesce(returns#20, 0.00) AS returns#26, (profit#23 - coalesce(profit_loss#21, 0.00)) AS profit#27] -(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] +(31) ColumnarToRow [codegen id : 1] +Input [5]: [channel#24, id#25, sales#22, returns#26, profit#27] + +(32) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#35), dynamicpruningexpression(cs_sold_date_sk#35 IN dynamicpruning#36)] +PartitionFilters: [isnotnull(cs_sold_date_sk#31), dynamicpruningexpression(cs_sold_date_sk#31 IN dynamicpruning#32)] ReadSchema: struct -(34) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#37] +(33) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#33] -(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) CometBroadcastHashJoin +Left output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] +Right output [1]: [d_date_sk#33] +Arguments: [cs_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight -(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) CometProject +Input [5]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31, d_date_sk#33] +Arguments: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30], [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] -(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))] +(36) CometHashAggregate +Input [3]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] +Keys [1]: [cs_call_center_sk#28] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#29)), partial_sum(UnscaledValue(cs_net_profit#30))] -(38) ColumnarToRow [codegen id : 5] -Input [3]: [cs_call_center_sk#32, sum#38, sum#39] +(37) CometColumnarExchange +Input [3]: [cs_call_center_sk#28, sum#34, sum#35] +Arguments: hashpartitioning(cs_call_center_sk#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(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] +(38) CometHashAggregate +Input [3]: [cs_call_center_sk#28, sum#34, sum#35] +Keys [1]: [cs_call_center_sk#28] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#29)), sum(UnscaledValue(cs_net_profit#30))] -(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] +(39) ColumnarToRow [codegen id : 2] +Input [3]: [cs_call_center_sk#28, sales#36, profit#37] -(41) BroadcastExchange -Input [3]: [cs_call_center_sk#32, sales#42, profit#43] -Arguments: IdentityBroadcastMode, [plan_id=5] +(40) BroadcastExchange +Input [3]: [cs_call_center_sk#28, sales#36, profit#37] +Arguments: IdentityBroadcastMode, [plan_id=4] -(42) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#44, cr_net_loss#45, cr_returned_date_sk#46] +(41) Scan parquet spark_catalog.default.catalog_returns +Output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#46), dynamicpruningexpression(cr_returned_date_sk#46 IN dynamicpruning#47)] +PartitionFilters: [isnotnull(cr_returned_date_sk#40), dynamicpruningexpression(cr_returned_date_sk#40 IN dynamicpruning#41)] ReadSchema: struct -(43) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#48] +(42) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#42] -(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) CometBroadcastHashJoin +Left output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] +Right output [1]: [d_date_sk#42] +Arguments: [cr_returned_date_sk#40], [d_date_sk#42], Inner, BuildRight -(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) CometProject +Input [4]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40, d_date_sk#42] +Arguments: [cr_return_amount#38, cr_net_loss#39], [cr_return_amount#38, cr_net_loss#39] -(46) CometHashAggregate -Input [2]: [cr_return_amount#44, cr_net_loss#45] +(45) CometHashAggregate +Input [2]: [cr_return_amount#38, cr_net_loss#39] Keys: [] -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] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#38)), partial_sum(UnscaledValue(cr_net_loss#39))] -(48) Exchange -Input [2]: [sum#49, sum#50] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +(46) CometColumnarExchange +Input [2]: [sum#43, sum#44] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(49) HashAggregate -Input [2]: [sum#49, sum#50] +(47) CometHashAggregate +Input [2]: [sum#43, sum#44] Keys: [] -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] +Functions [2]: [sum(UnscaledValue(cr_return_amount#38)), sum(UnscaledValue(cr_net_loss#39))] -(50) BroadcastNestedLoopJoin [codegen id : 8] +(48) ColumnarToRow +Input [2]: [returns#45, profit_loss#46] + +(49) BroadcastNestedLoopJoin [codegen id : 3] Join type: Inner Join condition: None -(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) Project [codegen id : 3] +Output [5]: [catalog channel AS channel#47, cs_call_center_sk#28 AS id#48, sales#36, returns#45, (profit#37 - profit_loss#46) AS profit#49] +Input [5]: [cs_call_center_sk#28, sales#36, profit#37, returns#45, profit_loss#46] -(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] +(51) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#61), dynamicpruningexpression(ws_sold_date_sk#61 IN dynamicpruning#62)] +PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct -(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) +(52) CometFilter +Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] +Condition : isnotnull(ws_web_page_sk#50) -(54) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#63] +(53) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#55] -(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 +(54) CometBroadcastHashJoin +Left output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] +Right output [1]: [d_date_sk#55] +Arguments: [ws_sold_date_sk#53], [d_date_sk#55], Inner, BuildRight -(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] +(55) CometProject +Input [5]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53, d_date_sk#55] +Arguments: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52], [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] -(57) Scan parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#64] +(56) Scan parquet spark_catalog.default.web_page +Output [1]: [wp_web_page_sk#56] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct -(58) CometFilter -Input [1]: [wp_web_page_sk#64] -Condition : isnotnull(wp_web_page_sk#64) - -(59) CometBroadcastExchange -Input [1]: [wp_web_page_sk#64] -Arguments: [wp_web_page_sk#64] +(57) CometFilter +Input [1]: [wp_web_page_sk#56] +Condition : isnotnull(wp_web_page_sk#56) -(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 +(58) CometBroadcastExchange +Input [1]: [wp_web_page_sk#56] +Arguments: [wp_web_page_sk#56] -(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] +(59) CometBroadcastHashJoin +Left output [3]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] +Right output [1]: [wp_web_page_sk#56] +Arguments: [ws_web_page_sk#50], [wp_web_page_sk#56], Inner, BuildRight -(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))] +(60) CometProject +Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] +Arguments: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56], [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] -(63) ColumnarToRow [codegen id : 9] -Input [3]: [wp_web_page_sk#64, sum#65, sum#66] +(61) CometHashAggregate +Input [3]: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] +Keys [1]: [wp_web_page_sk#56] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#51)), partial_sum(UnscaledValue(ws_net_profit#52))] -(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] +(62) CometColumnarExchange +Input [3]: [wp_web_page_sk#56, sum#57, sum#58] +Arguments: hashpartitioning(wp_web_page_sk#56, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(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] +(63) CometHashAggregate +Input [3]: [wp_web_page_sk#56, sum#57, sum#58] +Keys [1]: [wp_web_page_sk#56] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#51)), sum(UnscaledValue(ws_net_profit#52))] -(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] +(64) Scan parquet spark_catalog.default.web_returns +Output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#74), dynamicpruningexpression(wr_returned_date_sk#74 IN dynamicpruning#75)] +PartitionFilters: [isnotnull(wr_returned_date_sk#62), dynamicpruningexpression(wr_returned_date_sk#62 IN dynamicpruning#63)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct -(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) +(65) CometFilter +Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] +Condition : isnotnull(wr_web_page_sk#59) -(68) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#76] +(66) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#64] -(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 +(67) CometBroadcastHashJoin +Left output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] +Right output [1]: [d_date_sk#64] +Arguments: [wr_returned_date_sk#62], [d_date_sk#64], Inner, BuildRight -(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] +(68) CometProject +Input [5]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62, d_date_sk#64] +Arguments: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61], [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] -(71) ReusedExchange [Reuses operator id: 59] -Output [1]: [wp_web_page_sk#77] +(69) ReusedExchange [Reuses operator id: 58] +Output [1]: [wp_web_page_sk#65] -(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 +(70) CometBroadcastHashJoin +Left output [3]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] +Right output [1]: [wp_web_page_sk#65] +Arguments: [wr_web_page_sk#59], [wp_web_page_sk#65], 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] +(71) CometProject +Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] +Arguments: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65], [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] -(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 +(72) CometHashAggregate +Input [3]: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] +Keys [1]: [wp_web_page_sk#65] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#60)), partial_sum(UnscaledValue(wr_net_loss#61))] + +(73) CometColumnarExchange +Input [3]: [wp_web_page_sk#65, sum#66, sum#67] +Arguments: hashpartitioning(wp_web_page_sk#65, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(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] +(74) CometHashAggregate +Input [3]: [wp_web_page_sk#65, sum#66, sum#67] +Keys [1]: [wp_web_page_sk#65] +Functions [2]: [sum(UnscaledValue(wr_return_amt#60)), sum(UnscaledValue(wr_net_loss#61))] + +(75) CometBroadcastExchange +Input [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] +Arguments: [wp_web_page_sk#65, returns#68, profit_loss#69] + +(76) CometBroadcastHashJoin +Left output [3]: [wp_web_page_sk#56, sales#70, profit#71] +Right output [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] +Arguments: [wp_web_page_sk#56], [wp_web_page_sk#65], LeftOuter, BuildRight + +(77) CometProject +Input [6]: [wp_web_page_sk#56, sales#70, profit#71, wp_web_page_sk#65, returns#68, profit_loss#69] +Arguments: [channel#72, id#73, sales#70, returns#74, profit#75], [web channel AS channel#72, wp_web_page_sk#56 AS id#73, sales#70, coalesce(returns#68, 0.00) AS returns#74, (profit#71 - coalesce(profit_loss#69, 0.00)) AS profit#75] + +(78) ColumnarToRow [codegen id : 4] +Input [5]: [channel#72, id#73, sales#70, returns#74, profit#75] + +(79) Union + +(80) HashAggregate [codegen id : 5] +Input [5]: [channel#24, id#25, sales#22, returns#26, profit#27] +Keys [2]: [channel#24, id#25] +Functions [3]: [partial_sum(sales#22), partial_sum(returns#26), partial_sum(profit#27)] +Aggregate Attributes [6]: [sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81] +Results [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(81) RowToColumnar +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(82) CometColumnarExchange +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] +Arguments: hashpartitioning(channel#24, id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(83) ColumnarToRow [codegen id : 6] +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(84) HashAggregate [codegen id : 6] +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] +Keys [2]: [channel#24, id#25] +Functions [3]: [sum(sales#22), sum(returns#26), sum(profit#27)] +Aggregate Attributes [3]: [sum(sales#22)#88, sum(returns#26)#89, sum(profit#27)#90] +Results [5]: [channel#24, id#25, cast(sum(sales#22)#88 as decimal(37,2)) AS sales#91, cast(sum(returns#26)#89 as decimal(37,2)) AS returns#92, cast(sum(profit#27)#90 as decimal(38,2)) AS profit#93] + +(85) ReusedExchange [Reuses operator id: 82] +Output [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(86) ColumnarToRow [codegen id : 12] +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(87) HashAggregate [codegen id : 12] +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] +Keys [2]: [channel#24, id#25] +Functions [3]: [sum(sales#22), sum(returns#26), sum(profit#27)] +Aggregate Attributes [3]: [sum(sales#22)#88, sum(returns#26)#89, sum(profit#27)#90] +Results [4]: [channel#24, sum(sales#22)#88 AS sales#94, sum(returns#26)#89 AS returns#95, sum(profit#27)#90 AS profit#96] + +(88) HashAggregate [codegen id : 12] +Input [4]: [channel#24, sales#94, returns#95, profit#96] +Keys [1]: [channel#24] +Functions [3]: [partial_sum(sales#94), partial_sum(returns#95), partial_sum(profit#96)] +Aggregate Attributes [6]: [sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102] +Results [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] + +(89) RowToColumnar +Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] + +(90) CometColumnarExchange +Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] +Arguments: hashpartitioning(channel#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(91) ColumnarToRow [codegen id : 13] +Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] + +(92) HashAggregate [codegen id : 13] +Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] +Keys [1]: [channel#24] +Functions [3]: [sum(sales#94), sum(returns#95), sum(profit#96)] +Aggregate Attributes [3]: [sum(sales#94)#109, sum(returns#95)#110, sum(profit#96)#111] +Results [5]: [channel#24, null AS id#112, sum(sales#94)#109 AS sales#113, sum(returns#95)#110 AS returns#114, sum(profit#96)#111 AS profit#115] + +(93) ReusedExchange [Reuses operator id: 82] +Output [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(94) ColumnarToRow [codegen id : 19] +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(95) HashAggregate [codegen id : 19] +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] +Keys [2]: [channel#24, id#25] +Functions [3]: [sum(sales#22), sum(returns#26), sum(profit#27)] +Aggregate Attributes [3]: [sum(sales#22)#88, sum(returns#26)#89, sum(profit#27)#90] +Results [3]: [sum(sales#22)#88 AS sales#94, sum(returns#26)#89 AS returns#95, sum(profit#27)#90 AS profit#96] + +(96) HashAggregate [codegen id : 19] +Input [3]: [sales#94, returns#95, profit#96] Keys: [] -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] +Functions [3]: [partial_sum(sales#94), partial_sum(returns#95), partial_sum(profit#96)] +Aggregate Attributes [6]: [sum#116, isEmpty#117, sum#118, isEmpty#119, sum#120, isEmpty#121] +Results [6]: [sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] + +(97) RowToColumnar +Input [6]: [sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] + +(98) CometColumnarExchange +Input [6]: [sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(93) Exchange -Input [6]: [sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] +(99) ColumnarToRow [codegen id : 20] +Input [6]: [sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] -(94) HashAggregate [codegen id : 44] -Input [6]: [sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] +(100) HashAggregate [codegen id : 20] +Input [6]: [sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] Keys: [] -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] +Functions [3]: [sum(sales#94), sum(returns#95), sum(profit#96)] +Aggregate Attributes [3]: [sum(sales#94)#128, sum(returns#95)#129, sum(profit#96)#130] +Results [5]: [null AS channel#131, null AS id#132, sum(sales#94)#128 AS sales#133, sum(returns#95)#129 AS returns#134, sum(profit#96)#130 AS profit#135] -(95) Union +(101) Union -(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] +(102) HashAggregate [codegen id : 21] +Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] +Keys [5]: [channel#24, id#25, sales#91, returns#92, profit#93] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#28, id#29, sales#103, returns#104, profit#105] +Results [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -(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] +(103) RowToColumnar +Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -(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] +(104) CometColumnarExchange +Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] +Arguments: hashpartitioning(channel#24, id#25, sales#91, returns#92, profit#93, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] + +(105) CometHashAggregate +Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] +Keys [5]: [channel#24, id#25, sales#91, returns#92, profit#93] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#28, id#29, sales#103, returns#104, profit#105] -(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] +(106) CometTakeOrderedAndProject +Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#24 ASC NULLS FIRST,id#25 ASC NULLS FIRST], output=[channel#24,id#25,sales#91,returns#92,profit#93]), [channel#24, id#25, sales#91, returns#92, profit#93], 100, [channel#24 ASC NULLS FIRST, id#25 ASC NULLS FIRST], [channel#24, id#25, sales#91, returns#92, profit#93] + +(107) ColumnarToRow [codegen id : 22] +Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (104) -+- * ColumnarToRow (103) - +- CometProject (102) - +- CometFilter (101) - +- CometScan parquet spark_catalog.default.date_dim (100) +BroadcastExchange (112) ++- * ColumnarToRow (111) + +- CometProject (110) + +- CometFilter (109) + +- CometScan parquet spark_catalog.default.date_dim (108) -(100) Scan parquet spark_catalog.default.date_dim +(108) 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 -(101) CometFilter +(109) 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)) -(102) CometProject +(110) CometProject Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(103) ColumnarToRow [codegen id : 1] +(111) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(104) BroadcastExchange +(112) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -Subquery:2 Hosting operator id = 18 Hosting Expression = sr_returned_date_sk#18 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 17 Hosting Expression = sr_returned_date_sk#14 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#35 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#31 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 42 Hosting Expression = cr_returned_date_sk#46 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 41 Hosting Expression = cr_returned_date_sk#40 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 52 Hosting Expression = ws_sold_date_sk#61 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 51 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 66 Hosting Expression = wr_returned_date_sk#74 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 64 Hosting Expression = wr_returned_date_sk#62 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 34972d290..6e5506d68 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,60 +1,56 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (46) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Exchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (45) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - 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 (13) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [s_store_sk,sum,sum,ss_ext_sales_price,ss_net_profit] - CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] - CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_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 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] - InputAdapter - 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] #8 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter +WholeStageCodegen (22) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [channel,id,sales,returns,profit] + CometColumnarExchange [channel,id,sales,returns,profit] #1 + RowToColumnar + WholeStageCodegen (21) + HashAggregate [channel,id,sales,returns,profit] + InputAdapter + Union + WholeStageCodegen (6) + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel,id] #2 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [s_store_sk,returns,profit,profit_loss] [channel,id,sales,returns,profit] + CometBroadcastHashJoin [s_store_sk,sales,profit,s_store_sk,returns,profit_loss] + CometHashAggregate [s_store_sk,sales,profit,sum,sum,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit))] + CometColumnarExchange [s_store_sk] #3 + CometHashAggregate [s_store_sk,sum,sum,ss_ext_sales_price,ss_net_profit] + CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_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 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [s_store_sk] #6 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] + CometBroadcastExchange [s_store_sk,returns,profit_loss] #7 + CometHashAggregate [s_store_sk,returns,profit_loss,sum,sum,sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss))] + CometColumnarExchange [s_store_sk] #8 CometHashAggregate [s_store_sk,sum,sum,sr_return_amt,sr_net_loss] CometProject [sr_return_amt,sr_net_loss,s_store_sk] CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] @@ -65,66 +61,54 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedSubquery [d_date_sk] #1 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 #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] + WholeStageCodegen (3) + Project [cs_call_center_sk,sales,returns,profit,profit_loss] + BroadcastNestedLoopJoin InputAdapter - Exchange [cs_call_center_sk] #10 - WholeStageCodegen (5) + BroadcastExchange #9 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [cs_call_center_sk,sum,sum,cs_ext_sales_price,cs_net_profit] - CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,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 - 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 #11 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,cr_return_amount,cr_net_loss] - CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_return_amount,cr_net_loss,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 - 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] #12 - WholeStageCodegen (9) + CometHashAggregate [cs_call_center_sk,sales,profit,sum,sum,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] + CometColumnarExchange [cs_call_center_sk] #10 + CometHashAggregate [cs_call_center_sk,sum,sum,cs_ext_sales_price,cs_net_profit] + CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,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 + ReusedExchange [d_date_sk] #5 ColumnarToRow InputAdapter - CometHashAggregate [wp_web_page_sk,sum,sum,ws_ext_sales_price,ws_net_profit] - CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk] - CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_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 + CometHashAggregate [returns,profit_loss,sum,sum,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] + CometColumnarExchange #11 + CometHashAggregate [sum,sum,cr_return_amount,cr_net_loss] + CometProject [cr_return_amount,cr_net_loss] + CometBroadcastHashJoin [cr_return_amount,cr_net_loss,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 ReusedExchange [d_date_sk] #5 - CometBroadcastExchange [wp_web_page_sk] #13 - CometFilter [wp_web_page_sk] - CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] - InputAdapter - 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] #15 - WholeStageCodegen (10) - ColumnarToRow - InputAdapter + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometProject [wp_web_page_sk,returns,profit,profit_loss] [channel,id,sales,returns,profit] + CometBroadcastHashJoin [wp_web_page_sk,sales,profit,wp_web_page_sk,returns,profit_loss] + CometHashAggregate [wp_web_page_sk,sales,profit,sum,sum,sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit))] + CometColumnarExchange [wp_web_page_sk] #12 + CometHashAggregate [wp_web_page_sk,sum,sum,ws_ext_sales_price,ws_net_profit] + CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] + CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk] + CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_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 + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange [wp_web_page_sk] #13 + CometFilter [wp_web_page_sk] + CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] + CometBroadcastExchange [wp_web_page_sk,returns,profit_loss] #14 + CometHashAggregate [wp_web_page_sk,returns,profit_loss,sum,sum,sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss))] + CometColumnarExchange [wp_web_page_sk] #15 CometHashAggregate [wp_web_page_sk,sum,sum,wr_return_amt,wr_net_loss] CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] @@ -135,21 +119,27 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedSubquery [d_date_sk] #1 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] #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 (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 #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 - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + WholeStageCodegen (13) + HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel] #16 + RowToColumnar + WholeStageCodegen (12) + 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] + ColumnarToRow + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + WholeStageCodegen (20) + HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange #17 + RowToColumnar + WholeStageCodegen (19) + 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] + ColumnarToRow + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt index c7ee5b1c9..9304cb85f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt @@ -1,74 +1,71 @@ == Physical Plan == -TakeOrderedAndProject (70) -+- * Project (69) - +- * SortMergeJoin Inner (68) - :- * Project (45) - : +- * SortMergeJoin Inner (44) - : :- * Sort (21) - : : +- * HashAggregate (20) - : : +- Exchange (19) - : : +- * HashAggregate (18) - : : +- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Project (14) - : : : +- * Filter (13) - : : : +- * SortMergeJoin LeftOuter (12) - : : : :- * Sort (5) - : : : : +- Exchange (4) - : : : : +- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- * Sort (11) - : : : +- Exchange (10) - : : : +- * ColumnarToRow (9) - : : : +- CometProject (8) - : : : +- CometFilter (7) - : : : +- CometScan parquet spark_catalog.default.store_returns (6) - : : +- ReusedExchange (15) - : +- * Sort (43) - : +- * Filter (42) - : +- * HashAggregate (41) - : +- Exchange (40) - : +- * HashAggregate (39) - : +- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * Project (35) - : : +- * Filter (34) - : : +- * SortMergeJoin LeftOuter (33) - : : :- * Sort (26) - : : : +- Exchange (25) - : : : +- * ColumnarToRow (24) - : : : +- CometFilter (23) - : : : +- CometScan parquet spark_catalog.default.web_sales (22) - : : +- * Sort (32) - : : +- Exchange (31) - : : +- * ColumnarToRow (30) - : : +- CometProject (29) - : : +- CometFilter (28) - : : +- CometScan parquet spark_catalog.default.web_returns (27) - : +- ReusedExchange (36) - +- * Sort (67) - +- * Filter (66) - +- * HashAggregate (65) - +- Exchange (64) - +- * HashAggregate (63) - +- * Project (62) - +- * BroadcastHashJoin Inner BuildRight (61) - :- * Project (59) - : +- * Filter (58) - : +- * SortMergeJoin LeftOuter (57) - : :- * Sort (50) - : : +- Exchange (49) - : : +- * ColumnarToRow (48) - : : +- CometFilter (47) - : : +- CometScan parquet spark_catalog.default.catalog_sales (46) - : +- * Sort (56) - : +- Exchange (55) - : +- * ColumnarToRow (54) - : +- CometProject (53) - : +- CometFilter (52) - : +- CometScan parquet spark_catalog.default.catalog_returns (51) - +- ReusedExchange (60) +TakeOrderedAndProject (67) ++- * Project (66) + +- * ColumnarToRow (65) + +- CometSortMergeJoin (64) + :- CometProject (43) + : +- CometSortMergeJoin (42) + : :- CometSort (21) + : : +- CometHashAggregate (20) + : : +- CometColumnarExchange (19) + : : +- CometHashAggregate (18) + : : +- CometProject (17) + : : +- CometBroadcastHashJoin (16) + : : :- CometProject (12) + : : : +- CometFilter (11) + : : : +- CometSortMergeJoin (10) + : : : :- CometSort (4) + : : : : +- CometColumnarExchange (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometSort (9) + : : : +- CometColumnarExchange (8) + : : : +- CometProject (7) + : : : +- CometFilter (6) + : : : +- CometScan parquet spark_catalog.default.store_returns (5) + : : +- CometBroadcastExchange (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.date_dim (13) + : +- CometSort (41) + : +- CometFilter (40) + : +- CometHashAggregate (39) + : +- CometColumnarExchange (38) + : +- CometHashAggregate (37) + : +- CometProject (36) + : +- CometBroadcastHashJoin (35) + : :- CometProject (33) + : : +- CometFilter (32) + : : +- CometSortMergeJoin (31) + : : :- CometSort (25) + : : : +- CometColumnarExchange (24) + : : : +- CometFilter (23) + : : : +- CometScan parquet spark_catalog.default.web_sales (22) + : : +- CometSort (30) + : : +- CometColumnarExchange (29) + : : +- CometProject (28) + : : +- CometFilter (27) + : : +- CometScan parquet spark_catalog.default.web_returns (26) + : +- ReusedExchange (34) + +- CometSort (63) + +- CometFilter (62) + +- CometHashAggregate (61) + +- CometColumnarExchange (60) + +- CometHashAggregate (59) + +- CometProject (58) + +- CometBroadcastHashJoin (57) + :- CometProject (55) + : +- CometFilter (54) + : +- CometSortMergeJoin (53) + : :- CometSort (47) + : : +- CometColumnarExchange (46) + : : +- CometFilter (45) + : : +- CometScan parquet spark_catalog.default.catalog_sales (44) + : +- CometSort (52) + : +- CometColumnarExchange (51) + : +- CometProject (50) + : +- CometFilter (49) + : +- CometScan parquet spark_catalog.default.catalog_returns (48) + +- ReusedExchange (56) (1) Scan parquet spark_catalog.default.store_sales @@ -83,349 +80,326 @@ ReadSchema: struct -(7) CometFilter +(6) CometFilter Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] Condition : (isnotnull(sr_ticket_number#10) AND isnotnull(sr_item_sk#9)) -(8) CometProject +(7) CometProject Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_item_sk#9, sr_ticket_number#10] -(9) ColumnarToRow [codegen id : 3] -Input [2]: [sr_item_sk#9, sr_ticket_number#10] - -(10) Exchange +(8) CometColumnarExchange Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(11) Sort [codegen id : 4] +(9) CometSort Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST], false, 0 +Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST] -(12) SortMergeJoin [codegen id : 6] -Left keys [2]: [ss_ticket_number#3, ss_item_sk#1] -Right keys [2]: [sr_ticket_number#10, sr_item_sk#9] -Join type: LeftOuter -Join condition: None +(10) CometSortMergeJoin +Left output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] +Right output [2]: [sr_item_sk#9, sr_ticket_number#10] +Arguments: [ss_ticket_number#3, ss_item_sk#1], [sr_ticket_number#10, sr_item_sk#9], LeftOuter -(13) Filter [codegen id : 6] +(11) CometFilter Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] Condition : isnull(sr_ticket_number#10) -(14) Project [codegen id : 6] -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] +(12) CometProject Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -(15) ReusedExchange [Reuses operator id: 74] +(13) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#12, d_year#13] +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 + +(14) CometFilter +Input [2]: [d_date_sk#12, d_year#13] +Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) + +(15) CometBroadcastExchange +Input [2]: [d_date_sk#12, d_year#13] +Arguments: [d_date_sk#12, d_year#13] -(16) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] +Right output [2]: [d_date_sk#12, d_year#13] +Arguments: [ss_sold_date_sk#7], [d_date_sk#12], Inner, BuildRight -(17) Project [codegen id : 6] -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] +(17) CometProject Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] -(18) HashAggregate [codegen id : 6] +(18) CometHashAggregate Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesale_cost#5)), partial_sum(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [3]: [sum#14, sum#15, sum#16] -Results [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] -(19) Exchange -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] -Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(19) CometColumnarExchange +Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] +Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(20) HashAggregate [codegen id : 7] -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] +(20) CometHashAggregate +Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [3]: [sum(ss_quantity#4)#20, sum(UnscaledValue(ss_wholesale_cost#5))#21, sum(UnscaledValue(ss_sales_price#6))#22] -Results [6]: [d_year#13 AS ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, sum(ss_quantity#4)#20 AS ss_qty#24, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#5))#21,17,2) AS ss_wc#25, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#22,17,2) AS ss_sp#26] -(21) Sort [codegen id : 7] -Input [6]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] -Arguments: [ss_sold_year#23 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], false, 0 +(21) CometSort +Input [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] +Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20], [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST] (22) Scan parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] 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#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#28)] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (23) CometFilter -Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Condition : (isnotnull(ws_item_sk#27) AND isnotnull(ws_bill_customer_sk#28)) +Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] +Condition : (isnotnull(ws_item_sk#21) AND isnotnull(ws_bill_customer_sk#22)) -(24) ColumnarToRow [codegen id : 8] -Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +(24) CometColumnarExchange +Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] +Arguments: hashpartitioning(ws_order_number#23, ws_item_sk#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(25) Exchange -Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Arguments: hashpartitioning(ws_order_number#29, ws_item_sk#27, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(25) CometSort +Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] +Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_order_number#23 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST] -(26) Sort [codegen id : 9] -Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Arguments: [ws_order_number#29 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST], false, 0 - -(27) Scan parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] +(26) Scan parquet spark_catalog.default.web_returns +Output [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct -(28) CometFilter -Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] -Condition : (isnotnull(wr_order_number#36) AND isnotnull(wr_item_sk#35)) - -(29) CometProject -Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] -Arguments: [wr_item_sk#35, wr_order_number#36], [wr_item_sk#35, wr_order_number#36] - -(30) ColumnarToRow [codegen id : 10] -Input [2]: [wr_item_sk#35, wr_order_number#36] - -(31) Exchange -Input [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: hashpartitioning(wr_order_number#36, wr_item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(32) Sort [codegen id : 11] -Input [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: [wr_order_number#36 ASC NULLS FIRST, wr_item_sk#35 ASC NULLS FIRST], false, 0 - -(33) SortMergeJoin [codegen id : 13] -Left keys [2]: [ws_order_number#29, ws_item_sk#27] -Right keys [2]: [wr_order_number#36, wr_item_sk#35] -Join type: LeftOuter -Join condition: None - -(34) Filter [codegen id : 13] -Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] -Condition : isnull(wr_order_number#36) - -(35) Project [codegen id : 13] -Output [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] - -(36) ReusedExchange [Reuses operator id: 74] -Output [2]: [d_date_sk#38, d_year#39] - -(37) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#38] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 13] -Output [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] -Input [8]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, d_date_sk#38, d_year#39] - -(39) HashAggregate [codegen id : 13] -Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] -Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] -Functions [3]: [partial_sum(ws_quantity#30), partial_sum(UnscaledValue(ws_wholesale_cost#31)), partial_sum(UnscaledValue(ws_sales_price#32))] -Aggregate Attributes [3]: [sum#40, sum#41, sum#42] -Results [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] - -(40) Exchange -Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] -Arguments: hashpartitioning(d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, 5), ENSURE_REQUIREMENTS, [plan_id=6] - -(41) HashAggregate [codegen id : 14] -Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] -Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] -Functions [3]: [sum(ws_quantity#30), sum(UnscaledValue(ws_wholesale_cost#31)), sum(UnscaledValue(ws_sales_price#32))] -Aggregate Attributes [3]: [sum(ws_quantity#30)#46, sum(UnscaledValue(ws_wholesale_cost#31))#47, sum(UnscaledValue(ws_sales_price#32))#48] -Results [6]: [d_year#39 AS ws_sold_year#49, ws_item_sk#27, ws_bill_customer_sk#28 AS ws_customer_sk#50, sum(ws_quantity#30)#46 AS ws_qty#51, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#31))#47,17,2) AS ws_wc#52, MakeDecimal(sum(UnscaledValue(ws_sales_price#32))#48,17,2) AS ws_sp#53] - -(42) Filter [codegen id : 14] -Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] -Condition : (coalesce(ws_qty#51, 0) > 0) - -(43) Sort [codegen id : 14] -Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] -Arguments: [ws_sold_year#49 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST, ws_customer_sk#50 ASC NULLS FIRST], false, 0 - -(44) SortMergeJoin [codegen id : 15] -Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50] -Join type: Inner -Join condition: None - -(45) Project [codegen id : 15] -Output [9]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53] -Input [12]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] - -(46) Scan parquet spark_catalog.default.catalog_sales -Output [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +(27) CometFilter +Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] +Condition : (isnotnull(wr_order_number#30) AND isnotnull(wr_item_sk#29)) + +(28) CometProject +Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] +Arguments: [wr_item_sk#29, wr_order_number#30], [wr_item_sk#29, wr_order_number#30] + +(29) CometColumnarExchange +Input [2]: [wr_item_sk#29, wr_order_number#30] +Arguments: hashpartitioning(wr_order_number#30, wr_item_sk#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(30) CometSort +Input [2]: [wr_item_sk#29, wr_order_number#30] +Arguments: [wr_item_sk#29, wr_order_number#30], [wr_order_number#30 ASC NULLS FIRST, wr_item_sk#29 ASC NULLS FIRST] + +(31) CometSortMergeJoin +Left output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] +Right output [2]: [wr_item_sk#29, wr_order_number#30] +Arguments: [ws_order_number#23, ws_item_sk#21], [wr_order_number#30, wr_item_sk#29], LeftOuter + +(32) CometFilter +Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] +Condition : isnull(wr_order_number#30) + +(33) CometProject +Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] +Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] + +(34) ReusedExchange [Reuses operator id: 15] +Output [2]: [d_date_sk#32, d_year#33] + +(35) CometBroadcastHashJoin +Left output [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] +Right output [2]: [d_date_sk#32, d_year#33] +Arguments: [ws_sold_date_sk#27], [d_date_sk#32], Inner, BuildRight + +(36) CometProject +Input [8]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, d_date_sk#32, d_year#33] +Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] + +(37) CometHashAggregate +Input [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] +Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] +Functions [3]: [partial_sum(ws_quantity#24), partial_sum(UnscaledValue(ws_wholesale_cost#25)), partial_sum(UnscaledValue(ws_sales_price#26))] + +(38) CometColumnarExchange +Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] +Arguments: hashpartitioning(d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(39) CometHashAggregate +Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] +Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] +Functions [3]: [sum(ws_quantity#24), sum(UnscaledValue(ws_wholesale_cost#25)), sum(UnscaledValue(ws_sales_price#26))] + +(40) CometFilter +Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] +Condition : (coalesce(ws_qty#39, 0) > 0) + +(41) CometSort +Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] +Arguments: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41], [ws_sold_year#37 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST, ws_customer_sk#38 ASC NULLS FIRST] + +(42) CometSortMergeJoin +Left output [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] +Right output [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] +Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38], Inner + +(43) CometProject +Input [12]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] +Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41], [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] + +(44) Scan parquet spark_catalog.default.catalog_sales +Output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_sold_date_sk#60 IN dynamicpruning#61)] +PartitionFilters: [isnotnull(cs_sold_date_sk#48), dynamicpruningexpression(cs_sold_date_sk#48 IN dynamicpruning#49)] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(47) CometFilter -Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Condition : (isnotnull(cs_item_sk#55) AND isnotnull(cs_bill_customer_sk#54)) - -(48) ColumnarToRow [codegen id : 16] -Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +(45) CometFilter +Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] +Condition : (isnotnull(cs_item_sk#43) AND isnotnull(cs_bill_customer_sk#42)) -(49) Exchange -Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Arguments: hashpartitioning(cs_order_number#56, cs_item_sk#55, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(46) CometColumnarExchange +Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] +Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(50) Sort [codegen id : 17] -Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Arguments: [cs_order_number#56 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST], false, 0 +(47) CometSort +Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] +Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_order_number#44 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST] -(51) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] +(48) Scan parquet spark_catalog.default.catalog_returns +Output [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(52) CometFilter -Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] -Condition : (isnotnull(cr_order_number#63) AND isnotnull(cr_item_sk#62)) - -(53) CometProject -Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] -Arguments: [cr_item_sk#62, cr_order_number#63], [cr_item_sk#62, cr_order_number#63] - -(54) ColumnarToRow [codegen id : 18] -Input [2]: [cr_item_sk#62, cr_order_number#63] - -(55) Exchange -Input [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: hashpartitioning(cr_order_number#63, cr_item_sk#62, 5), ENSURE_REQUIREMENTS, [plan_id=8] - -(56) Sort [codegen id : 19] -Input [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: [cr_order_number#63 ASC NULLS FIRST, cr_item_sk#62 ASC NULLS FIRST], false, 0 - -(57) SortMergeJoin [codegen id : 21] -Left keys [2]: [cs_order_number#56, cs_item_sk#55] -Right keys [2]: [cr_order_number#63, cr_item_sk#62] -Join type: LeftOuter -Join condition: None - -(58) Filter [codegen id : 21] -Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] -Condition : isnull(cr_order_number#63) - -(59) Project [codegen id : 21] -Output [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] - -(60) ReusedExchange [Reuses operator id: 74] -Output [2]: [d_date_sk#65, d_year#66] - -(61) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [cs_sold_date_sk#60] -Right keys [1]: [d_date_sk#65] -Join type: Inner -Join condition: None - -(62) Project [codegen id : 21] -Output [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] -Input [8]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, d_date_sk#65, d_year#66] - -(63) HashAggregate [codegen id : 21] -Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] -Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] -Functions [3]: [partial_sum(cs_quantity#57), partial_sum(UnscaledValue(cs_wholesale_cost#58)), partial_sum(UnscaledValue(cs_sales_price#59))] -Aggregate Attributes [3]: [sum#67, sum#68, sum#69] -Results [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] - -(64) Exchange -Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] -Arguments: hashpartitioning(d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, 5), ENSURE_REQUIREMENTS, [plan_id=9] - -(65) HashAggregate [codegen id : 22] -Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] -Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] -Functions [3]: [sum(cs_quantity#57), sum(UnscaledValue(cs_wholesale_cost#58)), sum(UnscaledValue(cs_sales_price#59))] -Aggregate Attributes [3]: [sum(cs_quantity#57)#73, sum(UnscaledValue(cs_wholesale_cost#58))#74, sum(UnscaledValue(cs_sales_price#59))#75] -Results [6]: [d_year#66 AS cs_sold_year#76, cs_item_sk#55, cs_bill_customer_sk#54 AS cs_customer_sk#77, sum(cs_quantity#57)#73 AS cs_qty#78, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#58))#74,17,2) AS cs_wc#79, MakeDecimal(sum(UnscaledValue(cs_sales_price#59))#75,17,2) AS cs_sp#80] - -(66) Filter [codegen id : 22] -Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] -Condition : (coalesce(cs_qty#78, 0) > 0) - -(67) Sort [codegen id : 22] -Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] -Arguments: [cs_sold_year#76 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST, cs_customer_sk#77 ASC NULLS FIRST], false, 0 - -(68) SortMergeJoin [codegen id : 23] -Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77] -Join type: Inner -Join condition: None - -(69) Project [codegen id : 23] -Output [13]: [round((cast(ss_qty#24 as double) / cast(coalesce((ws_qty#51 + cs_qty#78), 1) as double)), 2) AS ratio#81, ss_qty#24 AS store_qty#82, ss_wc#25 AS store_wholesale_cost#83, ss_sp#26 AS store_sales_price#84, (coalesce(ws_qty#51, 0) + coalesce(cs_qty#78, 0)) AS other_chan_qty#85, (coalesce(ws_wc#52, 0.00) + coalesce(cs_wc#79, 0.00)) AS other_chan_wholesale_cost#86, (coalesce(ws_sp#53, 0.00) + coalesce(cs_sp#80, 0.00)) AS other_chan_sales_price#87, ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] -Input [15]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53, cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] - -(70) TakeOrderedAndProject -Input [13]: [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87, ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] -Arguments: 100, [ss_sold_year#23 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#24 DESC NULLS LAST, ss_wc#25 DESC NULLS LAST, ss_sp#26 DESC NULLS LAST, other_chan_qty#85 ASC NULLS FIRST, other_chan_wholesale_cost#86 ASC NULLS FIRST, other_chan_sales_price#87 ASC NULLS FIRST, ratio#81 ASC NULLS FIRST], [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87] +(49) CometFilter +Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] +Condition : (isnotnull(cr_order_number#51) AND isnotnull(cr_item_sk#50)) + +(50) CometProject +Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] +Arguments: [cr_item_sk#50, cr_order_number#51], [cr_item_sk#50, cr_order_number#51] + +(51) CometColumnarExchange +Input [2]: [cr_item_sk#50, cr_order_number#51] +Arguments: hashpartitioning(cr_order_number#51, cr_item_sk#50, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(52) CometSort +Input [2]: [cr_item_sk#50, cr_order_number#51] +Arguments: [cr_item_sk#50, cr_order_number#51], [cr_order_number#51 ASC NULLS FIRST, cr_item_sk#50 ASC NULLS FIRST] + +(53) CometSortMergeJoin +Left output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] +Right output [2]: [cr_item_sk#50, cr_order_number#51] +Arguments: [cs_order_number#44, cs_item_sk#43], [cr_order_number#51, cr_item_sk#50], LeftOuter + +(54) CometFilter +Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] +Condition : isnull(cr_order_number#51) + +(55) CometProject +Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] +Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] + +(56) ReusedExchange [Reuses operator id: 15] +Output [2]: [d_date_sk#53, d_year#54] + +(57) CometBroadcastHashJoin +Left output [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] +Right output [2]: [d_date_sk#53, d_year#54] +Arguments: [cs_sold_date_sk#48], [d_date_sk#53], Inner, BuildRight + +(58) CometProject +Input [8]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, d_date_sk#53, d_year#54] +Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] + +(59) CometHashAggregate +Input [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] +Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] +Functions [3]: [partial_sum(cs_quantity#45), partial_sum(UnscaledValue(cs_wholesale_cost#46)), partial_sum(UnscaledValue(cs_sales_price#47))] + +(60) CometColumnarExchange +Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] +Arguments: hashpartitioning(d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(61) CometHashAggregate +Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] +Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] +Functions [3]: [sum(cs_quantity#45), sum(UnscaledValue(cs_wholesale_cost#46)), sum(UnscaledValue(cs_sales_price#47))] + +(62) CometFilter +Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] +Condition : (coalesce(cs_qty#60, 0) > 0) + +(63) CometSort +Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] +Arguments: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62], [cs_sold_year#58 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST, cs_customer_sk#59 ASC NULLS FIRST] + +(64) CometSortMergeJoin +Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] +Right output [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] +Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59], Inner + +(65) ColumnarToRow [codegen id : 1] +Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] + +(66) Project [codegen id : 1] +Output [13]: [round((cast(ss_qty#18 as double) / cast(coalesce((ws_qty#39 + cs_qty#60), 1) as double)), 2) AS ratio#63, ss_qty#18 AS store_qty#64, ss_wc#19 AS store_wholesale_cost#65, ss_sp#20 AS store_sales_price#66, (coalesce(ws_qty#39, 0) + coalesce(cs_qty#60, 0)) AS other_chan_qty#67, (coalesce(ws_wc#40, 0.00) + coalesce(cs_wc#61, 0.00)) AS other_chan_wholesale_cost#68, (coalesce(ws_sp#41, 0.00) + coalesce(cs_sp#62, 0.00)) AS other_chan_sales_price#69, ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] +Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] + +(67) TakeOrderedAndProject +Input [13]: [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69, ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] +Arguments: 100, [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 DESC NULLS LAST, ss_sp#20 DESC NULLS LAST, other_chan_qty#67 ASC NULLS FIRST, other_chan_wholesale_cost#68 ASC NULLS FIRST, other_chan_sales_price#69 ASC NULLS FIRST, ratio#63 ASC NULLS FIRST], [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (74) -+- * ColumnarToRow (73) - +- CometFilter (72) - +- CometScan parquet spark_catalog.default.date_dim (71) +BroadcastExchange (71) ++- * ColumnarToRow (70) + +- CometFilter (69) + +- CometScan parquet spark_catalog.default.date_dim (68) -(71) Scan parquet spark_catalog.default.date_dim +(68) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#12, d_year#13] 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 -(72) CometFilter +(69) CometFilter Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(73) ColumnarToRow [codegen id : 1] +(70) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] -(74) BroadcastExchange +(71) BroadcastExchange Input [2]: [d_date_sk#12, d_year#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#48 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt index b8e6e0605..6f7cc2edf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt @@ -1,127 +1,78 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ratio,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (23) + WholeStageCodegen (1) Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp,ss_sold_year,ss_item_sk,ss_customer_sk] - SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,cs_sold_year,cs_item_sk,cs_customer_sk] + ColumnarToRow InputAdapter - WholeStageCodegen (15) - Project [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] - SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ws_sold_year,ws_item_sk,ws_customer_sk] - InputAdapter - WholeStageCodegen (7) - Sort [ss_sold_year,ss_item_sk,ss_customer_sk] - HashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] [sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price)),ss_sold_year,ss_qty,ss_wc,ss_sp,sum,sum,sum] - InputAdapter - Exchange [d_year,ss_item_sk,ss_customer_sk] #1 - WholeStageCodegen (6) - HashAggregate [d_year,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] [sum,sum,sum,sum,sum,sum] - Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - Filter [sr_ticket_number] - SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - WholeStageCodegen (2) - Sort [ss_ticket_number,ss_item_sk] - InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - WholeStageCodegen (4) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - Exchange [sr_ticket_number,sr_item_sk] #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - WholeStageCodegen (14) - Sort [ws_sold_year,ws_item_sk,ws_customer_sk] - Filter [ws_qty] - HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] [sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price)),ws_sold_year,ws_customer_sk,ws_qty,ws_wc,ws_sp,sum,sum,sum] - InputAdapter - Exchange [d_year,ws_item_sk,ws_bill_customer_sk] #5 - WholeStageCodegen (13) - HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] [sum,sum,sum,sum,sum,sum] - Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - Filter [wr_order_number] - SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - InputAdapter - WholeStageCodegen (9) - Sort [ws_order_number,ws_item_sk] - InputAdapter - Exchange [ws_order_number,ws_item_sk] #6 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - WholeStageCodegen (11) - Sort [wr_order_number,wr_item_sk] - InputAdapter - Exchange [wr_order_number,wr_item_sk] #7 - WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - WholeStageCodegen (22) - Sort [cs_sold_year,cs_item_sk,cs_customer_sk] - Filter [cs_qty] - HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] [sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price)),cs_sold_year,cs_customer_sk,cs_qty,cs_wc,cs_sp,sum,sum,sum] - InputAdapter - Exchange [d_year,cs_item_sk,cs_bill_customer_sk] #8 - WholeStageCodegen (21) - HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] [sum,sum,sum,sum,sum,sum] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - Filter [cr_order_number] - SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - InputAdapter - WholeStageCodegen (17) - Sort [cs_order_number,cs_item_sk] - InputAdapter - Exchange [cs_order_number,cs_item_sk] #9 - WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - WholeStageCodegen (19) - Sort [cr_order_number,cr_item_sk] - InputAdapter - Exchange [cr_order_number,cr_item_sk] #10 - WholeStageCodegen (18) + CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] + CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] + CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] + CometSort [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp] + CometHashAggregate [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,d_year,sum,sum,sum,sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price))] + CometColumnarExchange [d_year,ss_item_sk,ss_customer_sk] #1 + CometHashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum,ss_quantity,ss_wholesale_cost,ss_sales_price] + CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #2 + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometSort [sr_item_sk,sr_ticket_number] + CometColumnarExchange [sr_ticket_number,sr_item_sk] #4 + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #5 + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometSort [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] + CometFilter [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] + CometHashAggregate [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp,d_year,ws_bill_customer_sk,sum,sum,sum,sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price))] + CometColumnarExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 + CometHashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum,ws_quantity,ws_wholesale_cost,ws_sales_price] + CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] + CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,d_date_sk,d_year] + CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] + CometSortMergeJoin [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] + CometSort [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometColumnarExchange [ws_order_number,ws_item_sk] #7 + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_item_sk,wr_order_number] + CometColumnarExchange [wr_order_number,wr_item_sk] #8 + CometProject [wr_item_sk,wr_order_number] + CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] + ReusedExchange [d_date_sk,d_year] #5 + CometSort [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] + CometFilter [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] + CometHashAggregate [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp,d_year,cs_bill_customer_sk,sum,sum,sum,sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price))] + CometColumnarExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 + CometHashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum,cs_quantity,cs_wholesale_cost,cs_sales_price] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] + CometSortMergeJoin [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] + CometSort [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + CometColumnarExchange [cs_order_number,cs_item_sk] #10 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [cr_item_sk,cr_order_number] + CometColumnarExchange [cr_order_number,cr_item_sk] #11 + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] + ReusedExchange [d_date_sk,d_year] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt index 7cd1e2225..2e7e5aaf5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt @@ -1,124 +1,129 @@ == Physical Plan == -TakeOrderedAndProject (120) -+- * HashAggregate (119) - +- Exchange (118) - +- * HashAggregate (117) - +- Union (116) - :- * HashAggregate (105) - : +- Exchange (104) - : +- * HashAggregate (103) - : +- Union (102) - : :- * HashAggregate (39) - : : +- Exchange (38) - : : +- * HashAggregate (37) - : : +- * Project (36) - : : +- * BroadcastHashJoin Inner BuildRight (35) - : : :- * Project (29) - : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : :- * Project (22) - : : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : : :- * Project (16) - : : : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : : : :- * Project (13) - : : : : : : +- * SortMergeJoin LeftOuter (12) - : : : : : : :- * Sort (5) - : : : : : : : +- Exchange (4) - : : : : : : : +- * ColumnarToRow (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : +- * Sort (11) - : : : : : : +- Exchange (10) - : : : : : : +- * ColumnarToRow (9) - : : : : : : +- CometProject (8) - : : : : : : +- CometFilter (7) - : : : : : : +- CometScan parquet spark_catalog.default.store_returns (6) - : : : : : +- ReusedExchange (14) - : : : : +- BroadcastExchange (20) - : : : : +- * ColumnarToRow (19) - : : : : +- CometFilter (18) - : : : : +- CometScan parquet spark_catalog.default.store (17) - : : : +- BroadcastExchange (27) - : : : +- * ColumnarToRow (26) - : : : +- CometProject (25) - : : : +- CometFilter (24) - : : : +- CometScan parquet spark_catalog.default.item (23) - : : +- BroadcastExchange (34) - : : +- * ColumnarToRow (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan parquet spark_catalog.default.promotion (30) - : :- * HashAggregate (70) - : : +- Exchange (69) - : : +- * HashAggregate (68) - : : +- * Project (67) - : : +- * BroadcastHashJoin Inner BuildRight (66) - : : :- * Project (64) - : : : +- * BroadcastHashJoin Inner BuildRight (63) - : : : :- * Project (61) - : : : : +- * BroadcastHashJoin Inner BuildRight (60) - : : : : :- * Project (55) - : : : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : : : :- * Project (52) - : : : : : : +- * SortMergeJoin LeftOuter (51) - : : : : : : :- * Sort (44) - : : : : : : : +- Exchange (43) - : : : : : : : +- * ColumnarToRow (42) - : : : : : : : +- CometFilter (41) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (40) - : : : : : : +- * Sort (50) - : : : : : : +- Exchange (49) - : : : : : : +- * ColumnarToRow (48) - : : : : : : +- CometProject (47) - : : : : : : +- CometFilter (46) - : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (45) - : : : : : +- ReusedExchange (53) - : : : : +- BroadcastExchange (59) - : : : : +- * ColumnarToRow (58) - : : : : +- CometFilter (57) - : : : : +- CometScan parquet spark_catalog.default.catalog_page (56) - : : : +- ReusedExchange (62) - : : +- ReusedExchange (65) - : +- * HashAggregate (101) - : +- Exchange (100) - : +- * HashAggregate (99) - : +- * Project (98) - : +- * BroadcastHashJoin Inner BuildRight (97) - : :- * Project (95) - : : +- * BroadcastHashJoin Inner BuildRight (94) - : : :- * Project (92) - : : : +- * BroadcastHashJoin Inner BuildRight (91) - : : : :- * Project (86) - : : : : +- * BroadcastHashJoin Inner BuildRight (85) - : : : : :- * Project (83) - : : : : : +- * SortMergeJoin LeftOuter (82) - : : : : : :- * Sort (75) - : : : : : : +- Exchange (74) - : : : : : : +- * ColumnarToRow (73) - : : : : : : +- CometFilter (72) - : : : : : : +- CometScan parquet spark_catalog.default.web_sales (71) - : : : : : +- * Sort (81) - : : : : : +- Exchange (80) - : : : : : +- * ColumnarToRow (79) - : : : : : +- CometProject (78) - : : : : : +- CometFilter (77) - : : : : : +- CometScan parquet spark_catalog.default.web_returns (76) - : : : : +- ReusedExchange (84) - : : : +- BroadcastExchange (90) - : : : +- * ColumnarToRow (89) - : : : +- CometFilter (88) - : : : +- CometScan parquet spark_catalog.default.web_site (87) - : : +- ReusedExchange (93) - : +- ReusedExchange (96) - :- * HashAggregate (110) - : +- Exchange (109) - : +- * HashAggregate (108) - : +- * HashAggregate (107) - : +- ReusedExchange (106) - +- * HashAggregate (115) - +- Exchange (114) - +- * HashAggregate (113) - +- * HashAggregate (112) - +- ReusedExchange (111) +* ColumnarToRow (125) ++- CometTakeOrderedAndProject (124) + +- CometHashAggregate (123) + +- CometColumnarExchange (122) + +- RowToColumnar (121) + +- * HashAggregate (120) + +- Union (119) + :- * HashAggregate (102) + : +- * ColumnarToRow (101) + : +- CometColumnarExchange (100) + : +- RowToColumnar (99) + : +- * HashAggregate (98) + : +- Union (97) + : :- * HashAggregate (38) + : : +- * ColumnarToRow (37) + : : +- CometColumnarExchange (36) + : : +- CometHashAggregate (35) + : : +- CometProject (34) + : : +- CometBroadcastHashJoin (33) + : : :- CometProject (28) + : : : +- CometBroadcastHashJoin (27) + : : : :- CometProject (22) + : : : : +- CometBroadcastHashJoin (21) + : : : : :- CometProject (17) + : : : : : +- CometBroadcastHashJoin (16) + : : : : : :- CometProject (11) + : : : : : : +- CometSortMergeJoin (10) + : : : : : : :- CometSort (4) + : : : : : : : +- CometColumnarExchange (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- CometSort (9) + : : : : : : +- CometColumnarExchange (8) + : : : : : : +- CometProject (7) + : : : : : : +- CometFilter (6) + : : : : : : +- CometScan parquet spark_catalog.default.store_returns (5) + : : : : : +- CometBroadcastExchange (15) + : : : : : +- CometProject (14) + : : : : : +- CometFilter (13) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (12) + : : : : +- CometBroadcastExchange (20) + : : : : +- CometFilter (19) + : : : : +- CometScan parquet spark_catalog.default.store (18) + : : : +- CometBroadcastExchange (26) + : : : +- CometProject (25) + : : : +- CometFilter (24) + : : : +- CometScan parquet spark_catalog.default.item (23) + : : +- CometBroadcastExchange (32) + : : +- CometProject (31) + : : +- CometFilter (30) + : : +- CometScan parquet spark_catalog.default.promotion (29) + : :- * HashAggregate (67) + : : +- * ColumnarToRow (66) + : : +- CometColumnarExchange (65) + : : +- CometHashAggregate (64) + : : +- CometProject (63) + : : +- CometBroadcastHashJoin (62) + : : :- CometProject (60) + : : : +- CometBroadcastHashJoin (59) + : : : :- CometProject (57) + : : : : +- CometBroadcastHashJoin (56) + : : : : :- CometProject (52) + : : : : : +- CometBroadcastHashJoin (51) + : : : : : :- CometProject (49) + : : : : : : +- CometSortMergeJoin (48) + : : : : : : :- CometSort (42) + : : : : : : : +- CometColumnarExchange (41) + : : : : : : : +- CometFilter (40) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (39) + : : : : : : +- CometSort (47) + : : : : : : +- CometColumnarExchange (46) + : : : : : : +- CometProject (45) + : : : : : : +- CometFilter (44) + : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (43) + : : : : : +- ReusedExchange (50) + : : : : +- CometBroadcastExchange (55) + : : : : +- CometFilter (54) + : : : : +- CometScan parquet spark_catalog.default.catalog_page (53) + : : : +- ReusedExchange (58) + : : +- ReusedExchange (61) + : +- * HashAggregate (96) + : +- * ColumnarToRow (95) + : +- CometColumnarExchange (94) + : +- CometHashAggregate (93) + : +- CometProject (92) + : +- CometBroadcastHashJoin (91) + : :- CometProject (89) + : : +- CometBroadcastHashJoin (88) + : : :- CometProject (86) + : : : +- CometBroadcastHashJoin (85) + : : : :- CometProject (81) + : : : : +- CometBroadcastHashJoin (80) + : : : : :- CometProject (78) + : : : : : +- CometSortMergeJoin (77) + : : : : : :- CometSort (71) + : : : : : : +- CometColumnarExchange (70) + : : : : : : +- CometFilter (69) + : : : : : : +- CometScan parquet spark_catalog.default.web_sales (68) + : : : : : +- CometSort (76) + : : : : : +- CometColumnarExchange (75) + : : : : : +- CometProject (74) + : : : : : +- CometFilter (73) + : : : : : +- CometScan parquet spark_catalog.default.web_returns (72) + : : : : +- ReusedExchange (79) + : : : +- CometBroadcastExchange (84) + : : : +- CometFilter (83) + : : : +- CometScan parquet spark_catalog.default.web_site (82) + : : +- ReusedExchange (87) + : +- ReusedExchange (90) + :- * HashAggregate (110) + : +- * ColumnarToRow (109) + : +- CometColumnarExchange (108) + : +- RowToColumnar (107) + : +- * HashAggregate (106) + : +- * HashAggregate (105) + : +- * ColumnarToRow (104) + : +- ReusedExchange (103) + +- * HashAggregate (118) + +- * ColumnarToRow (117) + +- CometColumnarExchange (116) + +- RowToColumnar (115) + +- * HashAggregate (114) + +- * HashAggregate (113) + +- * ColumnarToRow (112) + +- ReusedExchange (111) (1) Scan parquet spark_catalog.default.store_sales @@ -133,598 +138,597 @@ ReadSchema: struct -(7) CometFilter +(6) CometFilter Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] Condition : (isnotnull(sr_item_sk#9) AND isnotnull(sr_ticket_number#10)) -(8) CometProject +(7) CometProject Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -(9) ColumnarToRow [codegen id : 3] -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] - -(10) Exchange +(8) CometColumnarExchange Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(11) Sort [codegen id : 4] +(9) CometSort Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST], false, 0 +Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST] -(12) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_item_sk#1, ss_ticket_number#4] -Right keys [2]: [sr_item_sk#9, sr_ticket_number#10] -Join type: LeftOuter -Join condition: None +(10) CometSortMergeJoin +Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] +Right output [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] +Arguments: [ss_item_sk#1, ss_ticket_number#4], [sr_item_sk#9, sr_ticket_number#10], LeftOuter -(13) Project [codegen id : 9] -Output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] +(11) CometProject Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] +Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] -(14) ReusedExchange [Reuses operator id: 125] -Output [1]: [d_date_sk#14] +(12) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#15] +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 -(15) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None +(13) CometFilter +Input [2]: [d_date_sk#14, d_date#15] +Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) -(16) Project [codegen id : 9] -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] +(14) CometProject +Input [2]: [d_date_sk#14, d_date#15] +Arguments: [d_date_sk#14], [d_date_sk#14] + +(15) CometBroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: [d_date_sk#14] + +(16) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] +Right output [1]: [d_date_sk#14] +Arguments: [ss_sold_date_sk#7], [d_date_sk#14], Inner, BuildRight + +(17) CometProject Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#14] +Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] -(17) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#15, s_store_id#16] +(18) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#16, s_store_id#17] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(18) CometFilter -Input [2]: [s_store_sk#15, s_store_id#16] -Condition : isnotnull(s_store_sk#15) - -(19) ColumnarToRow [codegen id : 6] -Input [2]: [s_store_sk#15, s_store_id#16] +(19) CometFilter +Input [2]: [s_store_sk#16, s_store_id#17] +Condition : isnotnull(s_store_sk#16) -(20) BroadcastExchange -Input [2]: [s_store_sk#15, s_store_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(20) CometBroadcastExchange +Input [2]: [s_store_sk#16, s_store_id#17] +Arguments: [s_store_sk#16, s_store_id#17] -(21) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#15] -Join type: Inner -Join condition: None +(21) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] +Right output [2]: [s_store_sk#16, s_store_id#17] +Arguments: [ss_store_sk#2], [s_store_sk#16], Inner, BuildRight -(22) Project [codegen id : 9] -Output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#15, s_store_id#16] +(22) CometProject +Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#16, s_store_id#17] +Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17], [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] (23) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#17, i_current_price#18] +Output [2]: [i_item_sk#18, i_current_price#19] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), GreaterThan(i_current_price,50.00), IsNotNull(i_item_sk)] ReadSchema: struct (24) CometFilter -Input [2]: [i_item_sk#17, i_current_price#18] -Condition : ((isnotnull(i_current_price#18) AND (i_current_price#18 > 50.00)) AND isnotnull(i_item_sk#17)) +Input [2]: [i_item_sk#18, i_current_price#19] +Condition : ((isnotnull(i_current_price#19) AND (i_current_price#19 > 50.00)) AND isnotnull(i_item_sk#18)) (25) CometProject -Input [2]: [i_item_sk#17, i_current_price#18] -Arguments: [i_item_sk#17], [i_item_sk#17] +Input [2]: [i_item_sk#18, i_current_price#19] +Arguments: [i_item_sk#18], [i_item_sk#18] -(26) ColumnarToRow [codegen id : 7] -Input [1]: [i_item_sk#17] +(26) CometBroadcastExchange +Input [1]: [i_item_sk#18] +Arguments: [i_item_sk#18] -(27) BroadcastExchange -Input [1]: [i_item_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(27) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] +Right output [1]: [i_item_sk#18] +Arguments: [ss_item_sk#1], [i_item_sk#18], Inner, BuildRight -(28) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#17] -Join type: Inner -Join condition: None +(28) CometProject +Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17, i_item_sk#18] +Arguments: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17], [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] -(29) Project [codegen id : 9] -Output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16, i_item_sk#17] - -(30) Scan parquet spark_catalog.default.promotion -Output [2]: [p_promo_sk#19, p_channel_tv#20] +(29) Scan parquet spark_catalog.default.promotion +Output [2]: [p_promo_sk#20, p_channel_tv#21] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_channel_tv), EqualTo(p_channel_tv,N), IsNotNull(p_promo_sk)] ReadSchema: struct -(31) CometFilter -Input [2]: [p_promo_sk#19, p_channel_tv#20] -Condition : ((isnotnull(p_channel_tv#20) AND (p_channel_tv#20 = N)) AND isnotnull(p_promo_sk#19)) - -(32) CometProject -Input [2]: [p_promo_sk#19, p_channel_tv#20] -Arguments: [p_promo_sk#19], [p_promo_sk#19] +(30) CometFilter +Input [2]: [p_promo_sk#20, p_channel_tv#21] +Condition : ((isnotnull(p_channel_tv#21) AND (p_channel_tv#21 = N)) AND isnotnull(p_promo_sk#20)) -(33) ColumnarToRow [codegen id : 8] -Input [1]: [p_promo_sk#19] +(31) CometProject +Input [2]: [p_promo_sk#20, p_channel_tv#21] +Arguments: [p_promo_sk#20], [p_promo_sk#20] -(34) BroadcastExchange -Input [1]: [p_promo_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(32) CometBroadcastExchange +Input [1]: [p_promo_sk#20] +Arguments: [p_promo_sk#20] -(35) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_promo_sk#3] -Right keys [1]: [p_promo_sk#19] -Join type: Inner -Join condition: None +(33) CometBroadcastHashJoin +Left output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] +Right output [1]: [p_promo_sk#20] +Arguments: [ss_promo_sk#3], [p_promo_sk#20], Inner, BuildRight -(36) Project [codegen id : 9] -Output [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] -Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16, p_promo_sk#19] +(34) CometProject +Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17, p_promo_sk#20] +Arguments: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17], [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] -(37) HashAggregate [codegen id : 9] -Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] -Keys [1]: [s_store_id#16] +(35) CometHashAggregate +Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] +Keys [1]: [s_store_id#17] Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] -Aggregate Attributes [5]: [sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -Results [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] -(38) Exchange -Input [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] -Arguments: hashpartitioning(s_store_id#16, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(36) CometColumnarExchange +Input [6]: [s_store_id#17, sum#22, sum#23, isEmpty#24, sum#25, isEmpty#26] +Arguments: hashpartitioning(s_store_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(39) HashAggregate [codegen id : 10] -Input [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] -Keys [1]: [s_store_id#16] +(37) ColumnarToRow [codegen id : 1] +Input [6]: [s_store_id#17, sum#22, sum#23, isEmpty#24, sum#25, isEmpty#26] + +(38) HashAggregate [codegen id : 1] +Input [6]: [s_store_id#17, sum#22, sum#23, isEmpty#24, sum#25, isEmpty#26] +Keys [1]: [s_store_id#17] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#5))#31, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#32, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#33] -Results [5]: [store channel AS channel#34, concat(store, s_store_id#16) AS id#35, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#31,17,2) AS sales#36, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#32 AS returns#37, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#33 AS profit#38] +Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#5))#27, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#28, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#29] +Results [5]: [store channel AS channel#30, concat(store, s_store_id#17) AS id#31, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#27,17,2) AS sales#32, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#28 AS returns#33, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#29 AS profit#34] -(40) Scan parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +(39) Scan parquet spark_catalog.default.catalog_sales +Output [7]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_order_number#38, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41] 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#41), dynamicpruningexpression(cs_sold_date_sk#41 IN dynamicpruning#42)] PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] ReadSchema: struct -(41) CometFilter -Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Condition : ((isnotnull(cs_catalog_page_sk#39) AND isnotnull(cs_item_sk#40)) AND isnotnull(cs_promo_sk#41)) - -(42) ColumnarToRow [codegen id : 11] -Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +(40) CometFilter +Input [7]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_order_number#38, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41] +Condition : ((isnotnull(cs_catalog_page_sk#35) AND isnotnull(cs_item_sk#36)) AND isnotnull(cs_promo_sk#37)) -(43) Exchange -Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Arguments: hashpartitioning(cs_item_sk#40, cs_order_number#42, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(41) CometColumnarExchange +Input [7]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_order_number#38, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41] +Arguments: hashpartitioning(cs_item_sk#36, cs_order_number#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(44) Sort [codegen id : 12] -Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Arguments: [cs_item_sk#40 ASC NULLS FIRST, cs_order_number#42 ASC NULLS FIRST], false, 0 +(42) CometSort +Input [7]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_order_number#38, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41] +Arguments: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_order_number#38, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41], [cs_item_sk#36 ASC NULLS FIRST, cs_order_number#38 ASC NULLS FIRST] -(45) Scan parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] +(43) Scan parquet spark_catalog.default.catalog_returns +Output [5]: [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46, cr_returned_date_sk#47] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(46) CometFilter -Input [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] -Condition : (isnotnull(cr_item_sk#47) AND isnotnull(cr_order_number#48)) - -(47) CometProject -Input [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] -Arguments: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50], [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] +(44) CometFilter +Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46, cr_returned_date_sk#47] +Condition : (isnotnull(cr_item_sk#43) AND isnotnull(cr_order_number#44)) -(48) ColumnarToRow [codegen id : 13] -Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] +(45) CometProject +Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46, cr_returned_date_sk#47] +Arguments: [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46], [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46] -(49) Exchange -Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -Arguments: hashpartitioning(cr_item_sk#47, cr_order_number#48, 5), ENSURE_REQUIREMENTS, [plan_id=8] +(46) CometColumnarExchange +Input [4]: [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46] +Arguments: hashpartitioning(cr_item_sk#43, cr_order_number#44, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(50) Sort [codegen id : 14] -Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -Arguments: [cr_item_sk#47 ASC NULLS FIRST, cr_order_number#48 ASC NULLS FIRST], false, 0 +(47) CometSort +Input [4]: [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46] +Arguments: [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46], [cr_item_sk#43 ASC NULLS FIRST, cr_order_number#44 ASC NULLS FIRST] -(51) SortMergeJoin [codegen id : 19] -Left keys [2]: [cs_item_sk#40, cs_order_number#42] -Right keys [2]: [cr_item_sk#47, cr_order_number#48] -Join type: LeftOuter -Join condition: None +(48) CometSortMergeJoin +Left output [7]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_order_number#38, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41] +Right output [4]: [cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46] +Arguments: [cs_item_sk#36, cs_order_number#38], [cr_item_sk#43, cr_order_number#44], LeftOuter -(52) Project [codegen id : 19] -Output [8]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50] -Input [11]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] +(49) CometProject +Input [11]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_order_number#38, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41, cr_item_sk#43, cr_order_number#44, cr_return_amount#45, cr_net_loss#46] +Arguments: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41, cr_return_amount#45, cr_net_loss#46], [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41, cr_return_amount#45, cr_net_loss#46] -(53) ReusedExchange [Reuses operator id: 125] -Output [1]: [d_date_sk#52] +(50) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#48] -(54) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_sold_date_sk#45] -Right keys [1]: [d_date_sk#52] -Join type: Inner -Join condition: None +(51) CometBroadcastHashJoin +Left output [8]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41, cr_return_amount#45, cr_net_loss#46] +Right output [1]: [d_date_sk#48] +Arguments: [cs_sold_date_sk#41], [d_date_sk#48], Inner, BuildRight -(55) Project [codegen id : 19] -Output [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50] -Input [9]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50, d_date_sk#52] +(52) CometProject +Input [9]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cs_sold_date_sk#41, cr_return_amount#45, cr_net_loss#46, d_date_sk#48] +Arguments: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46], [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46] -(56) Scan parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +(53) Scan parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#49, cp_catalog_page_id#50] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(57) CometFilter -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Condition : isnotnull(cp_catalog_page_sk#53) - -(58) ColumnarToRow [codegen id : 16] -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] - -(59) BroadcastExchange -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -(60) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_catalog_page_sk#39] -Right keys [1]: [cp_catalog_page_sk#53] -Join type: Inner -Join condition: None - -(61) Project [codegen id : 19] -Output [7]: [cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] -Input [9]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_sk#53, cp_catalog_page_id#54] - -(62) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#55] - -(63) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_item_sk#40] -Right keys [1]: [i_item_sk#55] -Join type: Inner -Join condition: None - -(64) Project [codegen id : 19] -Output [6]: [cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] -Input [8]: [cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54, i_item_sk#55] - -(65) ReusedExchange [Reuses operator id: 34] -Output [1]: [p_promo_sk#56] - -(66) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_promo_sk#41] -Right keys [1]: [p_promo_sk#56] -Join type: Inner -Join condition: None - -(67) Project [codegen id : 19] -Output [5]: [cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] -Input [7]: [cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54, p_promo_sk#56] - -(68) HashAggregate [codegen id : 19] -Input [5]: [cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] -Keys [1]: [cp_catalog_page_id#54] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#43)), partial_sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))] -Aggregate Attributes [5]: [sum#57, sum#58, isEmpty#59, sum#60, isEmpty#61] -Results [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] - -(69) Exchange -Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] -Arguments: hashpartitioning(cp_catalog_page_id#54, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(70) HashAggregate [codegen id : 20] -Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] -Keys [1]: [cp_catalog_page_id#54] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#43)), sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00)), sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#43))#67, sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00))#68, sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))#69] -Results [5]: [catalog channel AS channel#70, concat(catalog_page, cp_catalog_page_id#54) AS id#71, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#43))#67,17,2) AS sales#72, sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00))#68 AS returns#73, sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))#69 AS profit#74] - -(71) Scan parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] +(54) CometFilter +Input [2]: [cp_catalog_page_sk#49, cp_catalog_page_id#50] +Condition : isnotnull(cp_catalog_page_sk#49) + +(55) CometBroadcastExchange +Input [2]: [cp_catalog_page_sk#49, cp_catalog_page_id#50] +Arguments: [cp_catalog_page_sk#49, cp_catalog_page_id#50] + +(56) CometBroadcastHashJoin +Left output [7]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46] +Right output [2]: [cp_catalog_page_sk#49, cp_catalog_page_id#50] +Arguments: [cs_catalog_page_sk#35], [cp_catalog_page_sk#49], Inner, BuildRight + +(57) CometProject +Input [9]: [cs_catalog_page_sk#35, cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_sk#49, cp_catalog_page_id#50] +Arguments: [cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50], [cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50] + +(58) ReusedExchange [Reuses operator id: 26] +Output [1]: [i_item_sk#51] + +(59) CometBroadcastHashJoin +Left output [7]: [cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50] +Right output [1]: [i_item_sk#51] +Arguments: [cs_item_sk#36], [i_item_sk#51], Inner, BuildRight + +(60) CometProject +Input [8]: [cs_item_sk#36, cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50, i_item_sk#51] +Arguments: [cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50], [cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50] + +(61) ReusedExchange [Reuses operator id: 32] +Output [1]: [p_promo_sk#52] + +(62) CometBroadcastHashJoin +Left output [6]: [cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50] +Right output [1]: [p_promo_sk#52] +Arguments: [cs_promo_sk#37], [p_promo_sk#52], Inner, BuildRight + +(63) CometProject +Input [7]: [cs_promo_sk#37, cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50, p_promo_sk#52] +Arguments: [cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50], [cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50] + +(64) CometHashAggregate +Input [5]: [cs_ext_sales_price#39, cs_net_profit#40, cr_return_amount#45, cr_net_loss#46, cp_catalog_page_id#50] +Keys [1]: [cp_catalog_page_id#50] +Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#39)), partial_sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#40 - coalesce(cast(cr_net_loss#46 as decimal(12,2)), 0.00)))] + +(65) CometColumnarExchange +Input [6]: [cp_catalog_page_id#50, sum#53, sum#54, isEmpty#55, sum#56, isEmpty#57] +Arguments: hashpartitioning(cp_catalog_page_id#50, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(66) ColumnarToRow [codegen id : 2] +Input [6]: [cp_catalog_page_id#50, sum#53, sum#54, isEmpty#55, sum#56, isEmpty#57] + +(67) HashAggregate [codegen id : 2] +Input [6]: [cp_catalog_page_id#50, sum#53, sum#54, isEmpty#55, sum#56, isEmpty#57] +Keys [1]: [cp_catalog_page_id#50] +Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#39)), sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00)), sum((cs_net_profit#40 - coalesce(cast(cr_net_loss#46 as decimal(12,2)), 0.00)))] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#39))#58, sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00))#59, sum((cs_net_profit#40 - coalesce(cast(cr_net_loss#46 as decimal(12,2)), 0.00)))#60] +Results [5]: [catalog channel AS channel#61, concat(catalog_page, cp_catalog_page_id#50) AS id#62, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#39))#58,17,2) AS sales#63, sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00))#59 AS returns#64, sum((cs_net_profit#40 - coalesce(cast(cr_net_loss#46 as decimal(12,2)), 0.00)))#60 AS profit#65] + +(68) Scan parquet spark_catalog.default.web_sales +Output [7]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_order_number#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] 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#72), dynamicpruningexpression(ws_sold_date_sk#72 IN dynamicpruning#73)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct -(72) CometFilter -Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Condition : ((isnotnull(ws_web_site_sk#76) AND isnotnull(ws_item_sk#75)) AND isnotnull(ws_promo_sk#77)) - -(73) ColumnarToRow [codegen id : 21] -Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] +(69) CometFilter +Input [7]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_order_number#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Condition : ((isnotnull(ws_web_site_sk#67) AND isnotnull(ws_item_sk#66)) AND isnotnull(ws_promo_sk#68)) -(74) Exchange -Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Arguments: hashpartitioning(ws_item_sk#75, ws_order_number#78, 5), ENSURE_REQUIREMENTS, [plan_id=11] +(70) CometColumnarExchange +Input [7]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_order_number#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Arguments: hashpartitioning(ws_item_sk#66, ws_order_number#69, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(75) Sort [codegen id : 22] -Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Arguments: [ws_item_sk#75 ASC NULLS FIRST, ws_order_number#78 ASC NULLS FIRST], false, 0 +(71) CometSort +Input [7]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_order_number#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Arguments: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_order_number#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72], [ws_item_sk#66 ASC NULLS FIRST, ws_order_number#69 ASC NULLS FIRST] -(76) Scan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +(72) Scan parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77, wr_returned_date_sk#78] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct -(77) CometFilter -Input [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] -Condition : (isnotnull(wr_item_sk#83) AND isnotnull(wr_order_number#84)) +(73) CometFilter +Input [5]: [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77, wr_returned_date_sk#78] +Condition : (isnotnull(wr_item_sk#74) AND isnotnull(wr_order_number#75)) -(78) CometProject -Input [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] -Arguments: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86], [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] +(74) CometProject +Input [5]: [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77, wr_returned_date_sk#78] +Arguments: [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77], [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77] -(79) ColumnarToRow [codegen id : 23] -Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] +(75) CometColumnarExchange +Input [4]: [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77] +Arguments: hashpartitioning(wr_item_sk#74, wr_order_number#75, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(80) Exchange -Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -Arguments: hashpartitioning(wr_item_sk#83, wr_order_number#84, 5), ENSURE_REQUIREMENTS, [plan_id=12] +(76) CometSort +Input [4]: [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77] +Arguments: [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77], [wr_item_sk#74 ASC NULLS FIRST, wr_order_number#75 ASC NULLS FIRST] -(81) Sort [codegen id : 24] -Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -Arguments: [wr_item_sk#83 ASC NULLS FIRST, wr_order_number#84 ASC NULLS FIRST], false, 0 +(77) CometSortMergeJoin +Left output [7]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_order_number#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Right output [4]: [wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77] +Arguments: [ws_item_sk#66, ws_order_number#69], [wr_item_sk#74, wr_order_number#75], LeftOuter -(82) SortMergeJoin [codegen id : 29] -Left keys [2]: [ws_item_sk#75, ws_order_number#78] -Right keys [2]: [wr_item_sk#83, wr_order_number#84] -Join type: LeftOuter -Join condition: None - -(83) Project [codegen id : 29] -Output [8]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86] -Input [11]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] +(78) CometProject +Input [11]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_order_number#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, wr_item_sk#74, wr_order_number#75, wr_return_amt#76, wr_net_loss#77] +Arguments: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, wr_return_amt#76, wr_net_loss#77], [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, wr_return_amt#76, wr_net_loss#77] -(84) ReusedExchange [Reuses operator id: 125] -Output [1]: [d_date_sk#88] +(79) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#79] -(85) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_sold_date_sk#81] -Right keys [1]: [d_date_sk#88] -Join type: Inner -Join condition: None +(80) CometBroadcastHashJoin +Left output [8]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, wr_return_amt#76, wr_net_loss#77] +Right output [1]: [d_date_sk#79] +Arguments: [ws_sold_date_sk#72], [d_date_sk#79], Inner, BuildRight -(86) Project [codegen id : 29] -Output [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86] -Input [9]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86, d_date_sk#88] +(81) CometProject +Input [9]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, wr_return_amt#76, wr_net_loss#77, d_date_sk#79] +Arguments: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77], [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77] -(87) Scan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#89, web_site_id#90] +(82) Scan parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#80, web_site_id#81] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(88) CometFilter -Input [2]: [web_site_sk#89, web_site_id#90] -Condition : isnotnull(web_site_sk#89) - -(89) ColumnarToRow [codegen id : 26] -Input [2]: [web_site_sk#89, web_site_id#90] - -(90) BroadcastExchange -Input [2]: [web_site_sk#89, web_site_id#90] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] - -(91) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_web_site_sk#76] -Right keys [1]: [web_site_sk#89] -Join type: Inner -Join condition: None - -(92) Project [codegen id : 29] -Output [7]: [ws_item_sk#75, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] -Input [9]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_sk#89, web_site_id#90] - -(93) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#91] - -(94) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_item_sk#75] -Right keys [1]: [i_item_sk#91] -Join type: Inner -Join condition: None - -(95) Project [codegen id : 29] -Output [6]: [ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] -Input [8]: [ws_item_sk#75, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90, i_item_sk#91] - -(96) ReusedExchange [Reuses operator id: 34] -Output [1]: [p_promo_sk#92] - -(97) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_promo_sk#77] -Right keys [1]: [p_promo_sk#92] -Join type: Inner -Join condition: None - -(98) Project [codegen id : 29] -Output [5]: [ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] -Input [7]: [ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90, p_promo_sk#92] - -(99) HashAggregate [codegen id : 29] -Input [5]: [ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] -Keys [1]: [web_site_id#90] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#79)), partial_sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))] -Aggregate Attributes [5]: [sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] -Results [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] - -(100) Exchange -Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -Arguments: hashpartitioning(web_site_id#90, 5), ENSURE_REQUIREMENTS, [plan_id=14] - -(101) HashAggregate [codegen id : 30] -Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -Keys [1]: [web_site_id#90] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#79)), sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00)), sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#79))#103, sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00))#104, sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))#105] -Results [5]: [web channel AS channel#106, concat(web_site, web_site_id#90) AS id#107, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#79))#103,17,2) AS sales#108, sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00))#104 AS returns#109, sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))#105 AS profit#110] - -(102) Union - -(103) HashAggregate [codegen id : 31] -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#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116] -Results [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] - -(104) Exchange -Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] -Arguments: hashpartitioning(channel#34, id#35, 5), ENSURE_REQUIREMENTS, [plan_id=15] - -(105) HashAggregate [codegen id : 32] -Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] -Keys [2]: [channel#34, id#35] -Functions [3]: [sum(sales#36), sum(returns#37), sum(profit#38)] -Aggregate Attributes [3]: [sum(sales#36)#123, sum(returns#37)#124, sum(profit#38)#125] -Results [5]: [channel#34, id#35, cast(sum(sales#36)#123 as decimal(37,2)) AS sales#126, cast(sum(returns#37)#124 as decimal(38,2)) AS returns#127, cast(sum(profit#38)#125 as decimal(38,2)) AS profit#128] - -(106) ReusedExchange [Reuses operator id: 104] -Output [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] - -(107) HashAggregate [codegen id : 64] -Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] -Keys [2]: [channel#34, id#35] -Functions [3]: [sum(sales#36), sum(returns#37), sum(profit#38)] -Aggregate Attributes [3]: [sum(sales#36)#123, sum(returns#37)#124, sum(profit#38)#125] -Results [4]: [channel#34, sum(sales#36)#123 AS sales#129, sum(returns#37)#124 AS returns#130, sum(profit#38)#125 AS profit#131] - -(108) HashAggregate [codegen id : 64] -Input [4]: [channel#34, sales#129, returns#130, profit#131] -Keys [1]: [channel#34] -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#34, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] - -(109) Exchange -Input [7]: [channel#34, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] -Arguments: hashpartitioning(channel#34, 5), ENSURE_REQUIREMENTS, [plan_id=16] - -(110) HashAggregate [codegen id : 65] -Input [7]: [channel#34, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] -Keys [1]: [channel#34] -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#34, 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] - -(111) ReusedExchange [Reuses operator id: 104] -Output [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] - -(112) HashAggregate [codegen id : 97] -Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] -Keys [2]: [channel#34, id#35] -Functions [3]: [sum(sales#36), sum(returns#37), sum(profit#38)] -Aggregate Attributes [3]: [sum(sales#36)#123, sum(returns#37)#124, sum(profit#38)#125] -Results [3]: [sum(sales#36)#123 AS sales#129, sum(returns#37)#124 AS returns#130, sum(profit#38)#125 AS profit#131] - -(113) HashAggregate [codegen id : 97] -Input [3]: [sales#129, returns#130, profit#131] +(83) CometFilter +Input [2]: [web_site_sk#80, web_site_id#81] +Condition : isnotnull(web_site_sk#80) + +(84) CometBroadcastExchange +Input [2]: [web_site_sk#80, web_site_id#81] +Arguments: [web_site_sk#80, web_site_id#81] + +(85) CometBroadcastHashJoin +Left output [7]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77] +Right output [2]: [web_site_sk#80, web_site_id#81] +Arguments: [ws_web_site_sk#67], [web_site_sk#80], Inner, BuildRight + +(86) CometProject +Input [9]: [ws_item_sk#66, ws_web_site_sk#67, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_sk#80, web_site_id#81] +Arguments: [ws_item_sk#66, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81], [ws_item_sk#66, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81] + +(87) ReusedExchange [Reuses operator id: 26] +Output [1]: [i_item_sk#82] + +(88) CometBroadcastHashJoin +Left output [7]: [ws_item_sk#66, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81] +Right output [1]: [i_item_sk#82] +Arguments: [ws_item_sk#66], [i_item_sk#82], Inner, BuildRight + +(89) CometProject +Input [8]: [ws_item_sk#66, ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81, i_item_sk#82] +Arguments: [ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81], [ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81] + +(90) ReusedExchange [Reuses operator id: 32] +Output [1]: [p_promo_sk#83] + +(91) CometBroadcastHashJoin +Left output [6]: [ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81] +Right output [1]: [p_promo_sk#83] +Arguments: [ws_promo_sk#68], [p_promo_sk#83], Inner, BuildRight + +(92) CometProject +Input [7]: [ws_promo_sk#68, ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81, p_promo_sk#83] +Arguments: [ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81], [ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81] + +(93) CometHashAggregate +Input [5]: [ws_ext_sales_price#70, ws_net_profit#71, wr_return_amt#76, wr_net_loss#77, web_site_id#81] +Keys [1]: [web_site_id#81] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(coalesce(cast(wr_return_amt#76 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#71 - coalesce(cast(wr_net_loss#77 as decimal(12,2)), 0.00)))] + +(94) CometColumnarExchange +Input [6]: [web_site_id#81, sum#84, sum#85, isEmpty#86, sum#87, isEmpty#88] +Arguments: hashpartitioning(web_site_id#81, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(95) ColumnarToRow [codegen id : 3] +Input [6]: [web_site_id#81, sum#84, sum#85, isEmpty#86, sum#87, isEmpty#88] + +(96) HashAggregate [codegen id : 3] +Input [6]: [web_site_id#81, sum#84, sum#85, isEmpty#86, sum#87, isEmpty#88] +Keys [1]: [web_site_id#81] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#70)), sum(coalesce(cast(wr_return_amt#76 as decimal(12,2)), 0.00)), sum((ws_net_profit#71 - coalesce(cast(wr_net_loss#77 as decimal(12,2)), 0.00)))] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#70))#89, sum(coalesce(cast(wr_return_amt#76 as decimal(12,2)), 0.00))#90, sum((ws_net_profit#71 - coalesce(cast(wr_net_loss#77 as decimal(12,2)), 0.00)))#91] +Results [5]: [web channel AS channel#92, concat(web_site, web_site_id#81) AS id#93, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#70))#89,17,2) AS sales#94, sum(coalesce(cast(wr_return_amt#76 as decimal(12,2)), 0.00))#90 AS returns#95, sum((ws_net_profit#71 - coalesce(cast(wr_net_loss#77 as decimal(12,2)), 0.00)))#91 AS profit#96] + +(97) Union + +(98) HashAggregate [codegen id : 4] +Input [5]: [channel#30, id#31, sales#32, returns#33, profit#34] +Keys [2]: [channel#30, id#31] +Functions [3]: [partial_sum(sales#32), partial_sum(returns#33), partial_sum(profit#34)] +Aggregate Attributes [6]: [sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102] +Results [8]: [channel#30, id#31, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] + +(99) RowToColumnar +Input [8]: [channel#30, id#31, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] + +(100) CometColumnarExchange +Input [8]: [channel#30, id#31, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] +Arguments: hashpartitioning(channel#30, id#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(101) ColumnarToRow [codegen id : 5] +Input [8]: [channel#30, id#31, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] + +(102) HashAggregate [codegen id : 5] +Input [8]: [channel#30, id#31, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] +Keys [2]: [channel#30, id#31] +Functions [3]: [sum(sales#32), sum(returns#33), sum(profit#34)] +Aggregate Attributes [3]: [sum(sales#32)#109, sum(returns#33)#110, sum(profit#34)#111] +Results [5]: [channel#30, id#31, cast(sum(sales#32)#109 as decimal(37,2)) AS sales#112, cast(sum(returns#33)#110 as decimal(38,2)) AS returns#113, cast(sum(profit#34)#111 as decimal(38,2)) AS profit#114] + +(103) ReusedExchange [Reuses operator id: 100] +Output [8]: [channel#30, id#31, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] + +(104) ColumnarToRow [codegen id : 10] +Input [8]: [channel#30, id#31, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] + +(105) HashAggregate [codegen id : 10] +Input [8]: [channel#30, id#31, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] +Keys [2]: [channel#30, id#31] +Functions [3]: [sum(sales#32), sum(returns#33), sum(profit#34)] +Aggregate Attributes [3]: [sum(sales#32)#109, sum(returns#33)#110, sum(profit#34)#111] +Results [4]: [channel#30, sum(sales#32)#109 AS sales#115, sum(returns#33)#110 AS returns#116, sum(profit#34)#111 AS profit#117] + +(106) HashAggregate [codegen id : 10] +Input [4]: [channel#30, sales#115, returns#116, profit#117] +Keys [1]: [channel#30] +Functions [3]: [partial_sum(sales#115), partial_sum(returns#116), partial_sum(profit#117)] +Aggregate Attributes [6]: [sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123] +Results [7]: [channel#30, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] + +(107) RowToColumnar +Input [7]: [channel#30, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] + +(108) CometColumnarExchange +Input [7]: [channel#30, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] +Arguments: hashpartitioning(channel#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] + +(109) ColumnarToRow [codegen id : 11] +Input [7]: [channel#30, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] + +(110) HashAggregate [codegen id : 11] +Input [7]: [channel#30, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] +Keys [1]: [channel#30] +Functions [3]: [sum(sales#115), sum(returns#116), sum(profit#117)] +Aggregate Attributes [3]: [sum(sales#115)#130, sum(returns#116)#131, sum(profit#117)#132] +Results [5]: [channel#30, null AS id#133, sum(sales#115)#130 AS sales#134, sum(returns#116)#131 AS returns#135, sum(profit#117)#132 AS profit#136] + +(111) ReusedExchange [Reuses operator id: 100] +Output [8]: [channel#30, id#31, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] + +(112) ColumnarToRow [codegen id : 16] +Input [8]: [channel#30, id#31, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] + +(113) HashAggregate [codegen id : 16] +Input [8]: [channel#30, id#31, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] +Keys [2]: [channel#30, id#31] +Functions [3]: [sum(sales#32), sum(returns#33), sum(profit#34)] +Aggregate Attributes [3]: [sum(sales#32)#109, sum(returns#33)#110, sum(profit#34)#111] +Results [3]: [sum(sales#32)#109 AS sales#115, sum(returns#33)#110 AS returns#116, sum(profit#34)#111 AS profit#117] + +(114) HashAggregate [codegen id : 16] +Input [3]: [sales#115, returns#116, profit#117] Keys: [] -Functions [3]: [partial_sum(sales#129), partial_sum(returns#130), partial_sum(profit#131)] -Aggregate Attributes [6]: [sum#151, isEmpty#152, sum#153, isEmpty#154, sum#155, isEmpty#156] -Results [6]: [sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] +Functions [3]: [partial_sum(sales#115), partial_sum(returns#116), partial_sum(profit#117)] +Aggregate Attributes [6]: [sum#137, isEmpty#138, sum#139, isEmpty#140, sum#141, isEmpty#142] +Results [6]: [sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] -(114) Exchange -Input [6]: [sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=17] +(115) RowToColumnar +Input [6]: [sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] -(115) HashAggregate [codegen id : 98] -Input [6]: [sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] +(116) CometColumnarExchange +Input [6]: [sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] + +(117) ColumnarToRow [codegen id : 17] +Input [6]: [sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] + +(118) HashAggregate [codegen id : 17] +Input [6]: [sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] Keys: [] -Functions [3]: [sum(sales#129), sum(returns#130), sum(profit#131)] -Aggregate Attributes [3]: [sum(sales#129)#163, sum(returns#130)#164, sum(profit#131)#165] -Results [5]: [null AS channel#166, null AS id#167, sum(sales#129)#163 AS sales#168, sum(returns#130)#164 AS returns#169, sum(profit#131)#165 AS profit#170] +Functions [3]: [sum(sales#115), sum(returns#116), sum(profit#117)] +Aggregate Attributes [3]: [sum(sales#115)#149, sum(returns#116)#150, sum(profit#117)#151] +Results [5]: [null AS channel#152, null AS id#153, sum(sales#115)#149 AS sales#154, sum(returns#116)#150 AS returns#155, sum(profit#117)#151 AS profit#156] -(116) Union +(119) Union -(117) HashAggregate [codegen id : 99] -Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] -Keys [5]: [channel#34, id#35, sales#126, returns#127, profit#128] +(120) HashAggregate [codegen id : 18] +Input [5]: [channel#30, id#31, sales#112, returns#113, profit#114] +Keys [5]: [channel#30, id#31, sales#112, returns#113, profit#114] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#34, id#35, sales#126, returns#127, profit#128] +Results [5]: [channel#30, id#31, sales#112, returns#113, profit#114] + +(121) RowToColumnar +Input [5]: [channel#30, id#31, sales#112, returns#113, profit#114] -(118) Exchange -Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] -Arguments: hashpartitioning(channel#34, id#35, sales#126, returns#127, profit#128, 5), ENSURE_REQUIREMENTS, [plan_id=18] +(122) CometColumnarExchange +Input [5]: [channel#30, id#31, sales#112, returns#113, profit#114] +Arguments: hashpartitioning(channel#30, id#31, sales#112, returns#113, profit#114, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(119) HashAggregate [codegen id : 100] -Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] -Keys [5]: [channel#34, id#35, sales#126, returns#127, profit#128] +(123) CometHashAggregate +Input [5]: [channel#30, id#31, sales#112, returns#113, profit#114] +Keys [5]: [channel#30, id#31, sales#112, returns#113, profit#114] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#34, id#35, sales#126, returns#127, profit#128] -(120) TakeOrderedAndProject -Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] -Arguments: 100, [channel#34 ASC NULLS FIRST, id#35 ASC NULLS FIRST], [channel#34, id#35, sales#126, returns#127, profit#128] +(124) CometTakeOrderedAndProject +Input [5]: [channel#30, id#31, sales#112, returns#113, profit#114] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#30 ASC NULLS FIRST,id#31 ASC NULLS FIRST], output=[channel#30,id#31,sales#112,returns#113,profit#114]), [channel#30, id#31, sales#112, returns#113, profit#114], 100, [channel#30 ASC NULLS FIRST, id#31 ASC NULLS FIRST], [channel#30, id#31, sales#112, returns#113, profit#114] + +(125) ColumnarToRow [codegen id : 19] +Input [5]: [channel#30, id#31, sales#112, returns#113, profit#114] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (125) -+- * ColumnarToRow (124) - +- CometProject (123) - +- CometFilter (122) - +- CometScan parquet spark_catalog.default.date_dim (121) +BroadcastExchange (130) ++- * ColumnarToRow (129) + +- CometProject (128) + +- CometFilter (127) + +- CometScan parquet spark_catalog.default.date_dim (126) -(121) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#171] +(126) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#15] 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 -(122) CometFilter -Input [2]: [d_date_sk#14, d_date#171] -Condition : (((isnotnull(d_date#171) AND (d_date#171 >= 1998-08-04)) AND (d_date#171 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) +(127) CometFilter +Input [2]: [d_date_sk#14, d_date#15] +Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) -(123) CometProject -Input [2]: [d_date_sk#14, d_date#171] +(128) CometProject +Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(124) ColumnarToRow [codegen id : 1] +(129) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(125) BroadcastExchange +(130) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] -Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 71 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#72 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt index 0793067df..210c9b6d2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt @@ -1,207 +1,157 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (100) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Exchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (99) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (32) - 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 (31) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (10) - HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] +WholeStageCodegen (19) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [channel,id,sales,returns,profit] + CometColumnarExchange [channel,id,sales,returns,profit] #1 + RowToColumnar + WholeStageCodegen (18) + HashAggregate [channel,id,sales,returns,profit] + InputAdapter + Union + WholeStageCodegen (5) + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel,id] #2 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange [s_store_id] #3 - WholeStageCodegen (9) - HashAggregate [s_store_id,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - SortMergeJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - WholeStageCodegen (2) - Sort [ss_item_sk,ss_ticket_number] - InputAdapter - Exchange [ss_item_sk,ss_ticket_number] #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - WholeStageCodegen (4) - Sort [sr_item_sk,sr_ticket_number] - InputAdapter - Exchange [sr_item_sk,sr_ticket_number] #6 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter + Union + WholeStageCodegen (1) + HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [s_store_id] #3 + CometHashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] + CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] + CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] + CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] + CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk,ss_ticket_number] #4 + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometColumnarExchange [sr_item_sk,sr_ticket_number] #6 + CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #7 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [s_store_sk,s_store_id] #8 CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter + CometBroadcastExchange [i_item_sk] #9 CometProject [i_item_sk] CometFilter [i_item_sk,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter + CometBroadcastExchange [p_promo_sk] #10 CometProject [p_promo_sk] CometFilter [p_promo_sk,p_channel_tv] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - WholeStageCodegen (20) - HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [cp_catalog_page_id] #10 - WholeStageCodegen (19) - HashAggregate [cp_catalog_page_id,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_catalog_page_sk,cp_catalog_page_sk] - Project [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] - InputAdapter - WholeStageCodegen (12) - Sort [cs_item_sk,cs_order_number] - InputAdapter - Exchange [cs_item_sk,cs_order_number] #11 - WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - WholeStageCodegen (14) - Sort [cr_item_sk,cr_order_number] - InputAdapter - Exchange [cr_item_sk,cr_order_number] #12 - WholeStageCodegen (13) - ColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (16) - ColumnarToRow - InputAdapter + WholeStageCodegen (2) + HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [cp_catalog_page_id] #11 + CometHashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] + CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] + CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] + CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] + CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometColumnarExchange [cs_item_sk,cs_order_number] #12 + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometColumnarExchange [cr_item_sk,cr_order_number] #13 + CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #14 CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - InputAdapter - ReusedExchange [i_item_sk] #8 - InputAdapter - ReusedExchange [p_promo_sk] #9 - WholeStageCodegen (30) - HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [web_site_id] #14 - WholeStageCodegen (29) - HashAggregate [web_site_id,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_promo_sk,p_promo_sk] - Project [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - SortMergeJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] - InputAdapter - WholeStageCodegen (22) - Sort [ws_item_sk,ws_order_number] - InputAdapter - Exchange [ws_item_sk,ws_order_number] #15 - WholeStageCodegen (21) - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - WholeStageCodegen (24) - Sort [wr_item_sk,wr_order_number] - InputAdapter - Exchange [wr_item_sk,wr_order_number] #16 - WholeStageCodegen (23) - ColumnarToRow - InputAdapter - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (26) - ColumnarToRow - InputAdapter + ReusedExchange [i_item_sk] #9 + ReusedExchange [p_promo_sk] #10 + WholeStageCodegen (3) + HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [web_site_id] #15 + CometHashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] + CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] + CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] + CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] + CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometColumnarExchange [ws_item_sk,ws_order_number] #16 + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometColumnarExchange [wr_item_sk,wr_order_number] #17 + CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange [web_site_sk,web_site_id] #18 CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - InputAdapter - ReusedExchange [i_item_sk] #8 - InputAdapter - ReusedExchange [p_promo_sk] #9 - WholeStageCodegen (65) - 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] #18 - WholeStageCodegen (64) - 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 (98) - 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 #19 - WholeStageCodegen (97) - 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 - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + ReusedExchange [i_item_sk] #9 + ReusedExchange [p_promo_sk] #10 + WholeStageCodegen (11) + HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel] #19 + RowToColumnar + WholeStageCodegen (10) + 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] + ColumnarToRow + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + WholeStageCodegen (17) + HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange #20 + RowToColumnar + WholeStageCodegen (16) + 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] + ColumnarToRow + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 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 e125bab59..7a0d1b7ad 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 @@ -2,39 +2,39 @@ 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) + +- * ColumnarToRow (33) + +- CometSort (32) + +- CometColumnarExchange (31) + +- CometHashAggregate (30) + +- CometColumnarExchange (29) + +- CometHashAggregate (28) + +- CometUnion (27) + :- CometHashAggregate (16) + : +- CometColumnarExchange (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) + :- CometHashAggregate (21) + : +- CometColumnarExchange (20) + : +- CometHashAggregate (19) + : +- CometHashAggregate (18) + : +- ReusedExchange (17) + +- CometHashAggregate (26) + +- CometColumnarExchange (25) + +- CometHashAggregate (24) + +- CometHashAggregate (23) + +- ReusedExchange (22) (1) Scan parquet spark_catalog.default.web_sales @@ -106,115 +106,100 @@ 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))] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarExchange Input [3]: [i_category#9, i_class#8, sum#10] +Arguments: hashpartitioning(i_category#9, i_class#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(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] +(16) CometHashAggregate 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#9, i_class#8, 0 AS g_category#13, 0 AS g_class#14, 0 AS lochierarchy#15] -(18) ReusedExchange [Reuses operator id: 16] -Output [3]: [i_category#9, i_class#8, sum#16] +(17) ReusedExchange [Reuses operator id: 15] +Output [3]: [i_category#9, i_class#8, sum#11] -(19) HashAggregate [codegen id : 4] -Input [3]: [i_category#9, i_class#8, sum#16] +(18) CometHashAggregate +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 [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#11,17,2) AS total_sum#17, i_category#9] -(20) HashAggregate [codegen id : 4] -Input [2]: [total_sum#17, i_category#9] +(19) CometHashAggregate +Input [2]: [total_sum#12, 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#9, sum#20, isEmpty#21] +Functions [1]: [partial_sum(total_sum#12)] -(21) Exchange -Input [3]: [i_category#9, sum#20, isEmpty#21] -Arguments: hashpartitioning(i_category#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(20) CometColumnarExchange +Input [3]: [i_category#9, sum#13, isEmpty#14] +Arguments: hashpartitioning(i_category#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(22) HashAggregate [codegen id : 5] -Input [3]: [i_category#9, sum#20, isEmpty#21] +(21) CometHashAggregate +Input [3]: [i_category#9, sum#13, isEmpty#14] 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#9, null AS i_class#24, 0 AS g_category#25, 1 AS g_class#26, 1 AS lochierarchy#27] +Functions [1]: [sum(total_sum#12)] -(23) ReusedExchange [Reuses operator id: 16] -Output [3]: [i_category#9, i_class#8, sum#28] +(22) ReusedExchange [Reuses operator id: 15] +Output [3]: [i_category#9, i_class#8, sum#15] -(24) HashAggregate [codegen id : 7] -Input [3]: [i_category#9, i_class#8, sum#28] +(23) CometHashAggregate +Input [3]: [i_category#9, i_class#8, sum#15] 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] -(25) HashAggregate [codegen id : 7] -Input [1]: [total_sum#17] +(24) CometHashAggregate +Input [1]: [total_sum#12] Keys: [] -Functions [1]: [partial_sum(total_sum#17)] -Aggregate Attributes [2]: [sum#29, isEmpty#30] -Results [2]: [sum#31, isEmpty#32] +Functions [1]: [partial_sum(total_sum#12)] -(26) Exchange -Input [2]: [sum#31, isEmpty#32] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] +(25) CometColumnarExchange +Input [2]: [sum#16, isEmpty#17] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(27) HashAggregate [codegen id : 8] -Input [2]: [sum#31, isEmpty#32] +(26) CometHashAggregate +Input [2]: [sum#16, isEmpty#17] 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] +Functions [1]: [sum(total_sum#12)] -(28) Union +(27) CometUnion +Child 0 Input [6]: [total_sum#18, i_category#9, i_class#8, g_category#19, g_class#20, lochierarchy#21] +Child 1 Input [6]: [total_sum#22, i_category#9, i_class#23, g_category#24, g_class#25, lochierarchy#26] +Child 2 Input [6]: [total_sum#27, i_category#28, i_class#29, g_category#30, g_class#31, lochierarchy#32] -(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] +(28) CometHashAggregate +Input [6]: [total_sum#18, i_category#9, i_class#8, g_category#19, g_class#20, lochierarchy#21] +Keys [6]: [total_sum#18, i_category#9, i_class#8, g_category#19, g_class#20, lochierarchy#21] Functions: [] -Aggregate Attributes: [] -Results [6]: [total_sum#12, i_category#9, i_class#8, g_category#13, g_class#14, lochierarchy#15] -(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) CometColumnarExchange +Input [6]: [total_sum#18, i_category#9, i_class#8, g_category#19, g_class#20, lochierarchy#21] +Arguments: hashpartitioning(total_sum#18, i_category#9, i_class#8, g_category#19, g_class#20, lochierarchy#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(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] +(30) CometHashAggregate +Input [6]: [total_sum#18, i_category#9, i_class#8, g_category#19, g_class#20, lochierarchy#21] +Keys [6]: [total_sum#18, i_category#9, i_class#8, g_category#19, g_class#20, lochierarchy#21] Functions: [] -Aggregate Attributes: [] -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] -(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) CometColumnarExchange +Input [5]: [total_sum#18, i_category#9, i_class#8, lochierarchy#21, _w0#33] +Arguments: hashpartitioning(lochierarchy#21, _w0#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(32) CometSort +Input [5]: [total_sum#18, i_category#9, i_class#8, lochierarchy#21, _w0#33] +Arguments: [total_sum#18, i_category#9, i_class#8, lochierarchy#21, _w0#33], [lochierarchy#21 ASC NULLS FIRST, _w0#33 ASC NULLS FIRST, total_sum#18 DESC NULLS LAST] -(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 +(33) ColumnarToRow [codegen id : 1] +Input [5]: [total_sum#18, i_category#9, i_class#8, lochierarchy#21, _w0#33] (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] +Input [5]: [total_sum#18, i_category#9, i_class#8, lochierarchy#21, _w0#33] +Arguments: [rank(total_sum#18) windowspecdefinition(lochierarchy#21, _w0#33, total_sum#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#34], [lochierarchy#21, _w0#33], [total_sum#18 DESC NULLS LAST] -(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] +(35) Project [codegen id : 2] +Output [5]: [total_sum#18, i_category#9, i_class#8, lochierarchy#21, rank_within_parent#34] +Input [6]: [total_sum#18, i_category#9, i_class#8, lochierarchy#21, _w0#33, rank_within_parent#34] (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] +Input [5]: [total_sum#18, i_category#9, i_class#8, lochierarchy#21, rank_within_parent#34] +Arguments: 100, [lochierarchy#21 DESC NULLS LAST, CASE WHEN (lochierarchy#21 = 0) THEN i_category#9 END ASC NULLS FIRST, rank_within_parent#34 ASC NULLS FIRST], [total_sum#18, i_category#9, i_class#8, lochierarchy#21, rank_within_parent#34] ===== Subqueries ===== 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 5adee59be..255e4acc3 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,64 +1,48 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (12) + WholeStageCodegen (2) Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (11) - Sort [lochierarchy,_w0,total_sum] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Exchange [lochierarchy,_w0] #1 - 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 (9) - HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - InputAdapter - Union - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_category,i_class,sum,ws_net_paid] - CometProject [ws_net_paid,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometFilter [i_item_sk,i_class,i_category] - 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] #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 (8) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty] - InputAdapter - 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 - ReusedExchange [i_category,i_class,sum] #3 + CometSort [total_sum,i_category,i_class,lochierarchy,_w0] + CometColumnarExchange [lochierarchy,_w0] #1 + CometHashAggregate [total_sum,i_category,i_class,lochierarchy,_w0,g_category,g_class] + CometColumnarExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 + CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] + CometUnion [total_sum,i_category,i_class,g_category,g_class,lochierarchy] + CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,sum(UnscaledValue(ws_net_paid))] + CometColumnarExchange [i_category,i_class] #3 + CometHashAggregate [i_category,i_class,sum,ws_net_paid] + CometProject [ws_net_paid,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] + CometProject [ws_item_sk,ws_net_paid] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometFilter [i_item_sk,i_class,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty,sum(total_sum)] + CometColumnarExchange [i_category] #7 + CometHashAggregate [i_category,sum,isEmpty,total_sum] + CometHashAggregate [total_sum,i_category,i_class,sum,sum(UnscaledValue(ws_net_paid))] + ReusedExchange [i_category,i_class,sum] #3 + CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty,sum(total_sum)] + CometColumnarExchange #8 + CometHashAggregate [sum,isEmpty,total_sum] + CometHashAggregate [total_sum,i_category,i_class,sum,sum(UnscaledValue(ws_net_paid))] + ReusedExchange [i_category,i_class,sum] #3 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 799687a19..8b0facb13 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,27 +1,29 @@ == Physical Plan == -* 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) +* ColumnarToRow (25) ++- CometSort (24) + +- CometColumnarExchange (23) + +- RowToColumnar (22) + +- * Project (21) + +- Window (20) + +- * ColumnarToRow (19) + +- CometSort (18) + +- CometColumnarExchange (17) + +- CometHashAggregate (16) + +- CometColumnarExchange (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 @@ -93,73 +95,77 @@ Input [6]: [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, 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))] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarExchange 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, CometColumnarShuffle, [plan_id=1] -(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] +(16) CometHashAggregate 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] -(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=2] +(17) CometColumnarExchange +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(18) CometSort +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] +Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15], [i_class#9 ASC NULLS FIRST] -(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 +(19) ColumnarToRow [codegen id : 1] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] (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] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] +Arguments: [sum(_w0#15) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#16], [i_class#9] + +(21) Project [codegen id : 2] +Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, ((_w0#15 * 100) / _we0#16) AS revenueratio#17] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, _we0#16] + +(22) RowToColumnar +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#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#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] +(23) CometColumnarExchange +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] +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#17 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(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=3] +(24) CometSort +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] +Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17], [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#17 ASC NULLS FIRST] -(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 +(25) ColumnarToRow [codegen id : 3] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] ===== 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 (30) ++- * ColumnarToRow (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan parquet spark_catalog.default.date_dim (26) -(24) Scan parquet spark_catalog.default.date_dim +(26) 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 -(25) CometFilter +(27) 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)) -(26) CometProject +(28) CometProject Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(27) ColumnarToRow [codegen id : 1] +(29) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(28) BroadcastExchange +(30) BroadcastExchange Input [1]: [d_date_sk#11] 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 64b162608..cbc817472 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,41 +1,39 @@ -WholeStageCodegen (5) - Sort [i_category,i_class,i_item_id,i_item_desc,revenueratio] +WholeStageCodegen (3) + ColumnarToRow InputAdapter - Exchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - 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 (3) - Sort [i_class] - InputAdapter - Exchange [i_class] #2 - 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ss_ext_sales_price] - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_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 #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 + RowToColumnar + WholeStageCodegen (2) + Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometColumnarExchange [i_class] #2 + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ss_ext_sales_price] + CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_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 #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index b9c5c3ca8..3d4c872ce 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -1706,7 +1706,6 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "true", CometConf.COMET_ENABLED.key -> "true", CometConf.COMET_EXEC_ENABLED.key -> "true", - CometConf.COMET_SHUFFLE_ENFORCE_MODE_ENABLED.key -> "true", CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true", EXTENDED_EXPLAIN_PROVIDERS_KEY -> "org.apache.comet.ExtendedExplainInfo") { val table = "test" diff --git a/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala index 2acc562b5..58cc9be8b 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala @@ -551,7 +551,6 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { Seq(true, false).foreach { dictionaryEnabled => withSQLConf( SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "true", - CometConf.COMET_SHUFFLE_ENFORCE_MODE_ENABLED.key -> "true", CometConf.COMET_BATCH_SIZE.key -> batchSize.toString) { withParquetTable( (0 until numValues).map(i => (i, Random.nextInt() % numGroups)), @@ -578,7 +577,6 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { Seq(true, false).foreach { dictionaryEnabled => withSQLConf( SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "true", - CometConf.COMET_SHUFFLE_ENFORCE_MODE_ENABLED.key -> "true", CometConf.COMET_BATCH_SIZE.key -> batchSize.toString) { withTempPath { dir => val path = new Path(dir.toURI.toString, "test.parquet") @@ -619,7 +617,6 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { Seq(true, false).foreach { dictionaryEnabled => withSQLConf( SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "true", - CometConf.COMET_SHUFFLE_ENFORCE_MODE_ENABLED.key -> "true", CometConf.COMET_BATCH_SIZE.key -> batchSize.toString) { withTempPath { dir => val path = new Path(dir.toURI.toString, "test.parquet") @@ -969,7 +966,6 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { withSQLConf( SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "true", CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", - CometConf.COMET_SHUFFLE_ENFORCE_MODE_ENABLED.key -> "true", CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { Seq(true, false).foreach { dictionary => withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { diff --git a/spark/src/test/scala/org/apache/comet/exec/CometColumnarShuffleSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometColumnarShuffleSuite.scala index 78b4bbb91..a64c7ab4d 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometColumnarShuffleSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometColumnarShuffleSuite.scala @@ -109,25 +109,6 @@ abstract class CometColumnarShuffleSuite extends CometTestBase with AdaptiveSpar checkSparkAnswer(df) } - test("Disable Comet shuffle with AQE coalesce partitions enabled") { - Seq(true, false).foreach { coalescePartitionsEnabled => - withSQLConf( - CometConf.COMET_EXEC_ENABLED.key -> "true", - SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", - SQLConf.COALESCE_PARTITIONS_ENABLED.key -> coalescePartitionsEnabled.toString, - SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { - val df = sql( - "SELECT * FROM (SELECT * FROM testData WHERE key = 0) t1 FULL JOIN " + - "testData2 t2 ON t1.key = t2.a") - if (coalescePartitionsEnabled) { - checkShuffleAnswer(df, 0) - } else { - checkShuffleAnswer(df, 2) - } - } - } - } - test("columnar shuffle on nested struct including nulls") { Seq(10, 201).foreach { numPartitions => Seq("1.0", "10.0").foreach { ratio => diff --git a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala index 5ca9401c6..37f92d6d8 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -69,7 +69,6 @@ class CometExecSuite extends CometTestBase { SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false", CometConf.COMET_EXEC_ENABLED.key -> "true", - CometConf.COMET_SHUFFLE_ENFORCE_MODE_ENABLED.key -> "true", CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { val data1 = diff --git a/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQuerySuite.scala b/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQuerySuite.scala index 864f2c03e..0c43fc8dd 100644 --- a/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQuerySuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQuerySuite.scala @@ -188,7 +188,6 @@ class CometTPCDSQuerySuite conf.set(CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key, "true") conf.set(CometConf.COMET_EXEC_SHUFFLE_ENABLED.key, "true") conf.set(CometConf.COMET_MEMORY_OVERHEAD.key, "15g") - conf.set(CometConf.COMET_SHUFFLE_ENFORCE_MODE_ENABLED.key, "true") conf.set("spark.sql.adaptive.coalescePartitions.enabled", "true") conf.set(MEMORY_OFFHEAP_ENABLED.key, "true") conf.set(MEMORY_OFFHEAP_SIZE.key, "15g") diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometTPCDSMicroBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometTPCDSMicroBenchmark.scala index fb1070c6a..08b8de2b2 100644 --- a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometTPCDSMicroBenchmark.scala +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometTPCDSMicroBenchmark.scala @@ -116,7 +116,6 @@ object CometTPCDSMicroBenchmark extends CometTPCQueryBenchmarkBase { withSQLConf( CometConf.COMET_ENABLED.key -> "true", CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", - CometConf.COMET_SHUFFLE_ENFORCE_MODE_ENABLED.key -> "true", CometConf.COMET_SHUFFLE_MODE.key -> "auto", CometConf.COMET_REGEXP_ALLOW_INCOMPATIBLE.key -> "true", CometConf.COMET_EXPLAIN_NATIVE_ENABLED.key -> "true", diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometTPCQueryBenchmarkBase.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometTPCQueryBenchmarkBase.scala index 2d78d03a6..1700b7ed8 100644 --- a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometTPCQueryBenchmarkBase.scala +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometTPCQueryBenchmarkBase.scala @@ -76,7 +76,6 @@ trait CometTPCQueryBenchmarkBase extends SqlBasedBenchmark with CometTPCQueryBas CometConf.COMET_EXEC_ENABLED.key -> "true", CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true", CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", - CometConf.COMET_SHUFFLE_ENFORCE_MODE_ENABLED.key -> "true", CometConf.COMET_SHUFFLE_MODE.key -> "auto") { cometSpark.sql(queryString).noop() }